From d82489bb9cd934acd546a84700074d5e033d23e8 Mon Sep 17 00:00:00 2001 From: Lionel Liu Date: Thu, 8 Jun 2017 16:26:57 +0800 Subject: [PATCH 001/111] log persist --- .../measure/batch/persist/LoggerPersist.scala | 85 +++++++++++++++++++ .../batch/persist/PersistFactory.scala | 2 + 2 files changed, 87 insertions(+) create mode 100644 measure/src/main/scala/org/apache/griffin/measure/batch/persist/LoggerPersist.scala diff --git a/measure/src/main/scala/org/apache/griffin/measure/batch/persist/LoggerPersist.scala b/measure/src/main/scala/org/apache/griffin/measure/batch/persist/LoggerPersist.scala new file mode 100644 index 000000000..9b9d4ccf1 --- /dev/null +++ b/measure/src/main/scala/org/apache/griffin/measure/batch/persist/LoggerPersist.scala @@ -0,0 +1,85 @@ +/*- + * Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. + + */ +package org.apache.griffin.measure.batch.persist + +import java.util.Date + +import org.apache.griffin.measure.batch.result._ +import org.apache.griffin.measure.batch.utils.HdfsUtil +import org.apache.spark.rdd.RDD + +// persist result and data to hdfs +case class LoggerPersist(config: Map[String, Any], metricName: String, timeStamp: Long) extends Persist { + + val MaxLogLines = "max.log.lines" + + val maxLogLines = try { config.getOrElse(MaxLogLines, 100).toString.toInt } catch { case _ => 100 } + + def available(): Boolean = true + + def start(msg: String): Unit = { + info(s"${metricName} start") + } + def finish(): Unit = { + info(s"${metricName} finish") + } + + def result(rt: Long, result: Result): Unit = { + try { + val resStr = result match { + case ar: AccuracyResult => { + s"match percentage: ${ar.matchPercentage}\ntotal count: ${ar.getTotal}\nmiss count: ${ar.getMiss}, match count: ${ar.getMatch}" + } + case pr: ProfileResult => { + s"match percentage: ${pr.matchPercentage}\ntotal count: ${pr.getTotal}\nmiss count: ${pr.getMiss}, match count: ${pr.getMatch}" + } + case _ => { + s"result: ${result}" + } + } + info(s"${metricName} result: ${resStr}") + } catch { + case e: Throwable => error(e.getMessage) + } + } + + // need to avoid string too long + private def rddRecords(records: RDD[String]): Unit = { + try { + val recordCount = records.count.toInt + val count = if (maxLogLines < 0) recordCount else scala.math.min(maxLogLines, recordCount) + if (count > 0) { + val recordsArray = records.take(count) + recordsArray.foreach(println) + } + } catch { + case e: Throwable => error(e.getMessage) + } + } + + def missRecords(records: RDD[String]): Unit = { + info(s"${metricName} miss records: ") + rddRecords(records) + } + def matchRecords(records: RDD[String]): Unit = { + info(s"${metricName} match records: ") + rddRecords(records) + } + + def log(rt: Long, msg: String): Unit = { + info(s"${rt}: ${msg}") + } + +} diff --git a/measure/src/main/scala/org/apache/griffin/measure/batch/persist/PersistFactory.scala b/measure/src/main/scala/org/apache/griffin/measure/batch/persist/PersistFactory.scala index 7f48b2de5..c18db9223 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/batch/persist/PersistFactory.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/batch/persist/PersistFactory.scala @@ -23,6 +23,7 @@ case class PersistFactory(persistParams: Iterable[PersistParam], metricName: Str val HDFS_REGEX = """^(?i)hdfs$""".r val HTTP_REGEX = """^(?i)http$""".r + val LOG_REGEX = """^(?i)log$""".r def getPersists(timeStamp: Long): MultiPersists = { MultiPersists(persistParams.flatMap(param => getPersist(timeStamp, param))) @@ -34,6 +35,7 @@ case class PersistFactory(persistParams: Iterable[PersistParam], metricName: Str val persistTry = persistParam.persistType match { case HDFS_REGEX() => Try(HdfsPersist(persistConfig, metricName, timeStamp)) case HTTP_REGEX() => Try(HttpPersist(persistConfig, metricName, timeStamp)) + case LOG_REGEX() => Try(LoggerPersist(persistConfig, metricName, timeStamp)) case _ => throw new Exception("not supported persist type") } persistTry match { From c5a799372abd49209de9e5d9bacaf1e92ed0764e Mon Sep 17 00:00:00 2001 From: Lionel Liu Date: Thu, 8 Jun 2017 16:30:11 +0800 Subject: [PATCH 002/111] persist --- .../apache/griffin/measure/batch/persist/LoggerPersist.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/measure/src/main/scala/org/apache/griffin/measure/batch/persist/LoggerPersist.scala b/measure/src/main/scala/org/apache/griffin/measure/batch/persist/LoggerPersist.scala index 9b9d4ccf1..da909c0b4 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/batch/persist/LoggerPersist.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/batch/persist/LoggerPersist.scala @@ -49,7 +49,7 @@ case class LoggerPersist(config: Map[String, Any], metricName: String, timeStamp s"result: ${result}" } } - info(s"${metricName} result: ${resStr}") + info(s"${metricName} result: \n${resStr}") } catch { case e: Throwable => error(e.getMessage) } From db7a7eb5d63246f3d978fa1732781f1d48254bf8 Mon Sep 17 00:00:00 2001 From: Lionel Liu Date: Mon, 12 Jun 2017 19:51:58 +0800 Subject: [PATCH 003/111] refact --- measure/src/main/resources/config-old.json | 45 ------------------- .../src/main/resources/config-streaming.json | 33 ++++++++++++++ .../measure/{batch => }/Application.scala | 19 ++++---- .../{batch => }/algo/AccuracyAlgo.scala | 2 +- .../measure/{batch => }/algo/Algo.scala | 8 ++-- .../{batch => }/algo/MeasureType.scala | 2 +- .../{batch => }/algo/ProfileAlgo.scala | 2 +- .../batch}/BatchAccuracyAlgo.scala | 17 +++---- .../batch}/BatchProfileAlgo.scala | 17 +++---- .../{batch => }/algo/core/AccuracyCore.scala | 6 +-- .../{batch => }/algo/core/ProfileCore.scala | 6 +-- .../{batch => }/config/params/AllParam.scala | 6 +-- .../{batch => }/config/params/Param.scala | 2 +- .../config/params/env/CleanerParam.scala | 4 +- .../config/params/env/EnvParam.scala | 4 +- .../config/params/env/PersistParam.scala | 4 +- .../config/params/env/SparkParam.scala | 4 +- .../params/user/DataConnectorParam.scala | 4 +- .../params/user/EvaluateRuleParam.scala | 4 +- .../config/params/user/UserParam.scala | 4 +- .../config/reader/ParamFileReader.scala | 6 +-- .../config/reader/ParamHdfsFileReader.scala | 8 ++-- .../config/reader/ParamRawStringReader.scala | 6 +-- .../config/reader/ParamReader.scala | 6 +-- .../config/reader/ParamReaderFactory.scala | 2 +- .../config/validator/AllParamValidator.scala | 4 +- .../config/validator/ParamValidator.scala | 6 +-- .../connector/AvroDataConnector.scala | 8 ++-- .../{batch => }/connector/DataConnector.scala | 2 +- .../connector/DataConnectorFactory.scala | 8 ++-- .../connector/HiveDataConnector.scala | 6 +-- .../measure/{batch => }/log/Loggable.scala | 2 +- .../{batch => }/persist/HdfsPersist.scala | 6 +-- .../{batch => }/persist/HttpPersist.scala | 6 +-- .../{batch => }/persist/LoggerPersist.scala | 6 +-- .../{batch => }/persist/MultiPersists.scala | 6 +-- .../measure/{batch => }/persist/Persist.scala | 6 +-- .../{batch => }/persist/PersistFactory.scala | 4 +- .../{batch => }/result/AccuracyResult.scala | 2 +- .../{batch => }/result/ProfileResult.scala | 2 +- .../measure/{batch => }/result/Result.scala | 2 +- .../{batch => }/result/ResultInfo.scala | 2 +- .../{batch => }/rule/CalculationUtil.scala | 2 +- .../{batch => }/rule/ExprValueUtil.scala | 4 +- .../{batch => }/rule/RuleAnalyzer.scala | 4 +- .../{batch => }/rule/RuleFactory.scala | 8 ++-- .../measure/{batch => }/rule/RuleParser.scala | 4 +- .../rule/expr/AnalyzableExpr.scala | 2 +- .../{batch => }/rule/expr/Cacheable.scala | 2 +- .../{batch => }/rule/expr/Calculatable.scala | 2 +- .../rule/expr/DataSourceable.scala | 2 +- .../{batch => }/rule/expr/Describable.scala | 2 +- .../measure/{batch => }/rule/expr/Expr.scala | 2 +- .../{batch => }/rule/expr/ExprDescOnly.scala | 2 +- .../{batch => }/rule/expr/ExprIdCounter.scala | 2 +- .../{batch => }/rule/expr/FieldDescOnly.scala | 2 +- .../{batch => }/rule/expr/LiteralExpr.scala | 2 +- .../{batch => }/rule/expr/LogicalExpr.scala | 4 +- .../{batch => }/rule/expr/MathExpr.scala | 4 +- .../{batch => }/rule/expr/SelectExpr.scala | 2 +- .../{batch => }/rule/expr/StatementExpr.scala | 2 +- .../measure/{batch => }/utils/HdfsUtil.scala | 2 +- .../measure/{batch => }/utils/HttpUtil.scala | 2 +- .../measure/{batch => }/utils/JsonUtil.scala | 2 +- .../algo/BatchAccuracyAlgoTest.scala | 20 ++++----- .../algo/BatchProfileAlgoTest.scala | 20 ++++----- .../algo/core/AccuracyCoreTest.scala | 8 ++-- .../algo/core/ProfileCoreTest.scala | 8 ++-- .../reader/ParamRawStringReaderTest.scala | 4 +- .../validator/AllParamValidatorTest.scala | 4 +- .../{batch => }/persist/HdfsPersistTest.scala | 2 +- .../{batch => }/persist/HttpPersistTest.scala | 2 +- .../result/AccuracyResultTest.scala | 2 +- .../result/ProfileResultTest.scala | 2 +- .../{batch => }/rule/RuleAnalyzerTest.scala | 4 +- .../{batch => }/rule/RuleFactoryTest.scala | 4 +- .../{batch => }/rule/RuleParserTest.scala | 2 +- .../{batch => }/utils/JsonUtilTest.scala | 2 +- .../src/main/resources/sparkJob.properties | 2 +- 79 files changed, 218 insertions(+), 227 deletions(-) delete mode 100644 measure/src/main/resources/config-old.json create mode 100644 measure/src/main/resources/config-streaming.json rename measure/src/main/scala/org/apache/griffin/measure/{batch => }/Application.scala (85%) rename measure/src/main/scala/org/apache/griffin/measure/{batch => }/algo/AccuracyAlgo.scala (92%) rename measure/src/main/scala/org/apache/griffin/measure/{batch => }/algo/Algo.scala (75%) rename measure/src/main/scala/org/apache/griffin/measure/{batch => }/algo/MeasureType.scala (93%) rename measure/src/main/scala/org/apache/griffin/measure/{batch => }/algo/ProfileAlgo.scala (92%) rename measure/src/main/scala/org/apache/griffin/measure/{batch/algo => algo/batch}/BatchAccuracyAlgo.scala (93%) rename measure/src/main/scala/org/apache/griffin/measure/{batch/algo => algo/batch}/BatchProfileAlgo.scala (90%) rename measure/src/main/scala/org/apache/griffin/measure/{batch => }/algo/core/AccuracyCore.scala (95%) rename measure/src/main/scala/org/apache/griffin/measure/{batch => }/algo/core/ProfileCore.scala (93%) rename measure/src/main/scala/org/apache/griffin/measure/{batch => }/config/params/AllParam.scala (84%) rename measure/src/main/scala/org/apache/griffin/measure/{batch => }/config/params/Param.scala (91%) rename measure/src/main/scala/org/apache/griffin/measure/{batch => }/config/params/env/CleanerParam.scala (86%) rename measure/src/main/scala/org/apache/griffin/measure/{batch => }/config/params/env/EnvParam.scala (88%) rename measure/src/main/scala/org/apache/griffin/measure/{batch => }/config/params/env/PersistParam.scala (88%) rename measure/src/main/scala/org/apache/griffin/measure/{batch => }/config/params/env/SparkParam.scala (88%) rename measure/src/main/scala/org/apache/griffin/measure/{batch => }/config/params/user/DataConnectorParam.scala (89%) rename measure/src/main/scala/org/apache/griffin/measure/{batch => }/config/params/user/EvaluateRuleParam.scala (88%) rename measure/src/main/scala/org/apache/griffin/measure/{batch => }/config/params/user/UserParam.scala (90%) rename measure/src/main/scala/org/apache/griffin/measure/{batch => }/config/reader/ParamFileReader.scala (83%) rename measure/src/main/scala/org/apache/griffin/measure/{batch => }/config/reader/ParamHdfsFileReader.scala (79%) rename measure/src/main/scala/org/apache/griffin/measure/{batch => }/config/reader/ParamRawStringReader.scala (82%) rename measure/src/main/scala/org/apache/griffin/measure/{batch => }/config/reader/ParamReader.scala (80%) rename measure/src/main/scala/org/apache/griffin/measure/{batch => }/config/reader/ParamReaderFactory.scala (95%) rename measure/src/main/scala/org/apache/griffin/measure/{batch => }/config/validator/AllParamValidator.scala (86%) rename measure/src/main/scala/org/apache/griffin/measure/{batch => }/config/validator/ParamValidator.scala (79%) rename measure/src/main/scala/org/apache/griffin/measure/{batch => }/connector/AvroDataConnector.scala (93%) rename measure/src/main/scala/org/apache/griffin/measure/{batch => }/connector/DataConnector.scala (93%) rename measure/src/main/scala/org/apache/griffin/measure/{batch => }/connector/DataConnectorFactory.scala (86%) rename measure/src/main/scala/org/apache/griffin/measure/{batch => }/connector/HiveDataConnector.scala (95%) rename measure/src/main/scala/org/apache/griffin/measure/{batch => }/log/Loggable.scala (95%) rename measure/src/main/scala/org/apache/griffin/measure/{batch => }/persist/HdfsPersist.scala (96%) rename measure/src/main/scala/org/apache/griffin/measure/{batch => }/persist/HttpPersist.scala (92%) rename measure/src/main/scala/org/apache/griffin/measure/{batch => }/persist/LoggerPersist.scala (94%) rename measure/src/main/scala/org/apache/griffin/measure/{batch => }/persist/MultiPersists.scala (89%) rename measure/src/main/scala/org/apache/griffin/measure/{batch => }/persist/Persist.scala (86%) rename measure/src/main/scala/org/apache/griffin/measure/{batch => }/persist/PersistFactory.scala (93%) rename measure/src/main/scala/org/apache/griffin/measure/{batch => }/result/AccuracyResult.scala (95%) rename measure/src/main/scala/org/apache/griffin/measure/{batch => }/result/ProfileResult.scala (96%) rename measure/src/main/scala/org/apache/griffin/measure/{batch => }/result/Result.scala (93%) rename measure/src/main/scala/org/apache/griffin/measure/{batch => }/result/ResultInfo.scala (96%) rename measure/src/main/scala/org/apache/griffin/measure/{batch => }/rule/CalculationUtil.scala (99%) rename measure/src/main/scala/org/apache/griffin/measure/{batch => }/rule/ExprValueUtil.scala (97%) rename measure/src/main/scala/org/apache/griffin/measure/{batch => }/rule/RuleAnalyzer.scala (97%) rename measure/src/main/scala/org/apache/griffin/measure/{batch => }/rule/RuleFactory.scala (84%) rename measure/src/main/scala/org/apache/griffin/measure/{batch => }/rule/RuleParser.scala (99%) rename measure/src/main/scala/org/apache/griffin/measure/{batch => }/rule/expr/AnalyzableExpr.scala (93%) rename measure/src/main/scala/org/apache/griffin/measure/{batch => }/rule/expr/Cacheable.scala (95%) rename measure/src/main/scala/org/apache/griffin/measure/{batch => }/rule/expr/Calculatable.scala (92%) rename measure/src/main/scala/org/apache/griffin/measure/{batch => }/rule/expr/DataSourceable.scala (94%) rename measure/src/main/scala/org/apache/griffin/measure/{batch => }/rule/expr/Describable.scala (93%) rename measure/src/main/scala/org/apache/griffin/measure/{batch => }/rule/expr/Expr.scala (96%) rename measure/src/main/scala/org/apache/griffin/measure/{batch => }/rule/expr/ExprDescOnly.scala (95%) rename measure/src/main/scala/org/apache/griffin/measure/{batch => }/rule/expr/ExprIdCounter.scala (96%) rename measure/src/main/scala/org/apache/griffin/measure/{batch => }/rule/expr/FieldDescOnly.scala (97%) rename measure/src/main/scala/org/apache/griffin/measure/{batch => }/rule/expr/LiteralExpr.scala (98%) rename measure/src/main/scala/org/apache/griffin/measure/{batch => }/rule/expr/LogicalExpr.scala (98%) rename measure/src/main/scala/org/apache/griffin/measure/{batch => }/rule/expr/MathExpr.scala (96%) rename measure/src/main/scala/org/apache/griffin/measure/{batch => }/rule/expr/SelectExpr.scala (98%) rename measure/src/main/scala/org/apache/griffin/measure/{batch => }/rule/expr/StatementExpr.scala (97%) rename measure/src/main/scala/org/apache/griffin/measure/{batch => }/utils/HdfsUtil.scala (97%) rename measure/src/main/scala/org/apache/griffin/measure/{batch => }/utils/HttpUtil.scala (97%) rename measure/src/main/scala/org/apache/griffin/measure/{batch => }/utils/JsonUtil.scala (96%) rename measure/src/test/scala/org/apache/griffin/measure/{batch => }/algo/BatchAccuracyAlgoTest.scala (91%) rename measure/src/test/scala/org/apache/griffin/measure/{batch => }/algo/BatchProfileAlgoTest.scala (88%) rename measure/src/test/scala/org/apache/griffin/measure/{batch => }/algo/core/AccuracyCoreTest.scala (92%) rename measure/src/test/scala/org/apache/griffin/measure/{batch => }/algo/core/ProfileCoreTest.scala (91%) rename measure/src/test/scala/org/apache/griffin/measure/{batch => }/config/reader/ParamRawStringReaderTest.scala (90%) rename measure/src/test/scala/org/apache/griffin/measure/{batch => }/config/validator/AllParamValidatorTest.scala (90%) rename measure/src/test/scala/org/apache/griffin/measure/{batch => }/persist/HdfsPersistTest.scala (96%) rename measure/src/test/scala/org/apache/griffin/measure/{batch => }/persist/HttpPersistTest.scala (96%) rename measure/src/test/scala/org/apache/griffin/measure/{batch => }/result/AccuracyResultTest.scala (97%) rename measure/src/test/scala/org/apache/griffin/measure/{batch => }/result/ProfileResultTest.scala (97%) rename measure/src/test/scala/org/apache/griffin/measure/{batch => }/rule/RuleAnalyzerTest.scala (95%) rename measure/src/test/scala/org/apache/griffin/measure/{batch => }/rule/RuleFactoryTest.scala (92%) rename measure/src/test/scala/org/apache/griffin/measure/{batch => }/rule/RuleParserTest.scala (99%) rename measure/src/test/scala/org/apache/griffin/measure/{batch => }/utils/JsonUtilTest.scala (97%) diff --git a/measure/src/main/resources/config-old.json b/measure/src/main/resources/config-old.json deleted file mode 100644 index 63dee698f..000000000 --- a/measure/src/main/resources/config-old.json +++ /dev/null @@ -1,45 +0,0 @@ -{ - "name": "accu1", - "type": "accuracy", - - "source": { - "connector": { - "type": "hive", - "version": "1.2", - "config": { - "table.name": "users_info_src", - "partitions": "dt=20170410, hour=14" - } - } - }, - - "target": { - "connector": { - "type": "hive", - "version": "1.2", - "config": { - "database": "default", - "table.name": "users_info_target", - "partitions": "dt=20170410, hour=14; dt=20170410, hour=15" - } - } - }, - - "evaluateRule": { - "sampleRatio": 1, - "assertion": { - "type": "DSL-griffin", - "rules": [ - { - "rule": "@Key ${source}['user_id'] === ${target}['user_id']" - }, - { - "rule": "${source}['first_name'] === ${target}['first_name']; ${source}['last_name'] === ${target}['last_name']; ${source}['address'] === ${target}['address']" - }, - { - "rule": "${source}['email'] === ${target}['email']; ${source}['phone'] === ${target}['phone']; ${source}['post_code'] === ${target}['post_code']" - } - ] - } - } -} \ No newline at end of file diff --git a/measure/src/main/resources/config-streaming.json b/measure/src/main/resources/config-streaming.json new file mode 100644 index 000000000..fe8877736 --- /dev/null +++ b/measure/src/main/resources/config-streaming.json @@ -0,0 +1,33 @@ +{ + "name": "accu2", + "type": "accuracy", + + "source": { + "type": "kafka", + "version": "0.8", + "config": { + "bootstrap.servers": "a.b.c.d:9092", + "group.id": "group1", + "auto.offset.reset": "smallest", + "auto.commit.enable": "false", + "topics": "src" + } + }, + + "target": { + "type": "kafka", + "version": "0.8", + "config": { + "bootstrap.servers": "a.b.c.d:9092", + "group.id": "group1", + "auto.offset.reset": "smallest", + "auto.commit.enable": "false", + "topics": "tgt" + } + }, + + "evaluateRule": { + "sampleRatio": 0.2, + "rules": "$source.json().seeds[*].json().url = $target.json().groups[0].attrsList['name' = 'URL'].values[0] AND $source.json().seeds[*].json().metadata.json().tracker.crawlRequestCreateTS = $target.json().groups[0].attrsList['name' = 'CRAWLMETADATA'].values[0].json().tracker.crawlRequestCreateTS WHEN $source._timestamp_ + 24h < $target._timestamp_" + } +} \ No newline at end of file diff --git a/measure/src/main/scala/org/apache/griffin/measure/batch/Application.scala b/measure/src/main/scala/org/apache/griffin/measure/Application.scala similarity index 85% rename from measure/src/main/scala/org/apache/griffin/measure/batch/Application.scala rename to measure/src/main/scala/org/apache/griffin/measure/Application.scala index 78f927198..a882fcea5 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/batch/Application.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/Application.scala @@ -12,15 +12,16 @@ See the License for the specific language governing permissions and limitations under the License. */ -package org.apache.griffin.measure.batch - -import org.apache.griffin.measure.batch.algo._ -import org.apache.griffin.measure.batch.config.params._ -import org.apache.griffin.measure.batch.config.params.env._ -import org.apache.griffin.measure.batch.config.params.user._ -import org.apache.griffin.measure.batch.config.reader._ -import org.apache.griffin.measure.batch.config.validator.AllParamValidator -import org.apache.griffin.measure.batch.log.Loggable +package org.apache.griffin.measure + +import org.apache.griffin.measure.algo._ +import org.apache.griffin.measure.algo.batch.{BatchAccuracyAlgo, BatchProfileAlgo} +import org.apache.griffin.measure.config.params._ +import org.apache.griffin.measure.config.params.env._ +import org.apache.griffin.measure.config.params.user._ +import org.apache.griffin.measure.config.reader._ +import org.apache.griffin.measure.config.validator.AllParamValidator +import org.apache.griffin.measure.log.Loggable import scala.util.{Failure, Success, Try} diff --git a/measure/src/main/scala/org/apache/griffin/measure/batch/algo/AccuracyAlgo.scala b/measure/src/main/scala/org/apache/griffin/measure/algo/AccuracyAlgo.scala similarity index 92% rename from measure/src/main/scala/org/apache/griffin/measure/batch/algo/AccuracyAlgo.scala rename to measure/src/main/scala/org/apache/griffin/measure/algo/AccuracyAlgo.scala index 16c65b0b6..1c2c1e574 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/batch/algo/AccuracyAlgo.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/algo/AccuracyAlgo.scala @@ -12,7 +12,7 @@ See the License for the specific language governing permissions and limitations under the License. */ -package org.apache.griffin.measure.batch.algo +package org.apache.griffin.measure.algo trait AccuracyAlgo extends Algo { diff --git a/measure/src/main/scala/org/apache/griffin/measure/batch/algo/Algo.scala b/measure/src/main/scala/org/apache/griffin/measure/algo/Algo.scala similarity index 75% rename from measure/src/main/scala/org/apache/griffin/measure/batch/algo/Algo.scala rename to measure/src/main/scala/org/apache/griffin/measure/algo/Algo.scala index 9e19a1872..c68c63572 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/batch/algo/Algo.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/algo/Algo.scala @@ -12,11 +12,11 @@ See the License for the specific language governing permissions and limitations under the License. */ -package org.apache.griffin.measure.batch.algo +package org.apache.griffin.measure.algo -import org.apache.griffin.measure.batch.config.params.env._ -import org.apache.griffin.measure.batch.config.params.user._ -import org.apache.griffin.measure.batch.log.Loggable +import org.apache.griffin.measure.config.params.env._ +import org.apache.griffin.measure.config.params.user._ +import org.apache.griffin.measure.log.Loggable import scala.util.Try diff --git a/measure/src/main/scala/org/apache/griffin/measure/batch/algo/MeasureType.scala b/measure/src/main/scala/org/apache/griffin/measure/algo/MeasureType.scala similarity index 93% rename from measure/src/main/scala/org/apache/griffin/measure/batch/algo/MeasureType.scala rename to measure/src/main/scala/org/apache/griffin/measure/algo/MeasureType.scala index aa14ac7b0..e827633d8 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/batch/algo/MeasureType.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/algo/MeasureType.scala @@ -12,7 +12,7 @@ See the License for the specific language governing permissions and limitations under the License. */ -package org.apache.griffin.measure.batch.algo +package org.apache.griffin.measure.algo object MeasureType { diff --git a/measure/src/main/scala/org/apache/griffin/measure/batch/algo/ProfileAlgo.scala b/measure/src/main/scala/org/apache/griffin/measure/algo/ProfileAlgo.scala similarity index 92% rename from measure/src/main/scala/org/apache/griffin/measure/batch/algo/ProfileAlgo.scala rename to measure/src/main/scala/org/apache/griffin/measure/algo/ProfileAlgo.scala index 418cce096..9108949dd 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/batch/algo/ProfileAlgo.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/algo/ProfileAlgo.scala @@ -12,7 +12,7 @@ See the License for the specific language governing permissions and limitations under the License. */ -package org.apache.griffin.measure.batch.algo +package org.apache.griffin.measure.algo trait ProfileAlgo extends Algo { diff --git a/measure/src/main/scala/org/apache/griffin/measure/batch/algo/BatchAccuracyAlgo.scala b/measure/src/main/scala/org/apache/griffin/measure/algo/batch/BatchAccuracyAlgo.scala similarity index 93% rename from measure/src/main/scala/org/apache/griffin/measure/batch/algo/BatchAccuracyAlgo.scala rename to measure/src/main/scala/org/apache/griffin/measure/algo/batch/BatchAccuracyAlgo.scala index 216c7c6a7..8debc0d82 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/batch/algo/BatchAccuracyAlgo.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/algo/batch/BatchAccuracyAlgo.scala @@ -12,17 +12,18 @@ See the License for the specific language governing permissions and limitations under the License. */ -package org.apache.griffin.measure.batch.algo +package org.apache.griffin.measure.algo.batch import java.util.Date -import org.apache.griffin.measure.batch.algo.core.AccuracyCore -import org.apache.griffin.measure.batch.config.params.AllParam -import org.apache.griffin.measure.batch.connector._ -import org.apache.griffin.measure.batch.rule._ -import org.apache.griffin.measure.batch.rule.expr._ -import org.apache.griffin.measure.batch.persist._ -import org.apache.griffin.measure.batch.result._ +import org.apache.griffin.measure.algo.AccuracyAlgo +import org.apache.griffin.measure.algo.core.AccuracyCore +import org.apache.griffin.measure.config.params.AllParam +import org.apache.griffin.measure.connector._ +import org.apache.griffin.measure.persist._ +import org.apache.griffin.measure.result._ +import org.apache.griffin.measure.rule._ +import org.apache.griffin.measure.rule.expr._ import org.apache.spark.rdd.RDD import org.apache.spark.sql.hive.HiveContext import org.apache.spark.{SparkConf, SparkContext} diff --git a/measure/src/main/scala/org/apache/griffin/measure/batch/algo/BatchProfileAlgo.scala b/measure/src/main/scala/org/apache/griffin/measure/algo/batch/BatchProfileAlgo.scala similarity index 90% rename from measure/src/main/scala/org/apache/griffin/measure/batch/algo/BatchProfileAlgo.scala rename to measure/src/main/scala/org/apache/griffin/measure/algo/batch/BatchProfileAlgo.scala index 9c0f48b89..645459ec4 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/batch/algo/BatchProfileAlgo.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/algo/batch/BatchProfileAlgo.scala @@ -12,17 +12,18 @@ See the License for the specific language governing permissions and limitations under the License. */ -package org.apache.griffin.measure.batch.algo +package org.apache.griffin.measure.algo.batch import java.util.Date -import org.apache.griffin.measure.batch.algo.core.ProfileCore -import org.apache.griffin.measure.batch.config.params._ -import org.apache.griffin.measure.batch.connector.{DataConnector, DataConnectorFactory} -import org.apache.griffin.measure.batch.persist.{Persist, PersistFactory} -import org.apache.griffin.measure.batch.result._ -import org.apache.griffin.measure.batch.rule.{ExprValueUtil, RuleAnalyzer, RuleFactory} -import org.apache.griffin.measure.batch.rule.expr.{Expr, StatementExpr} +import org.apache.griffin.measure.algo.ProfileAlgo +import org.apache.griffin.measure.algo.core.ProfileCore +import org.apache.griffin.measure.config.params._ +import org.apache.griffin.measure.connector.{DataConnector, DataConnectorFactory} +import org.apache.griffin.measure.persist.{Persist, PersistFactory} +import org.apache.griffin.measure.result._ +import org.apache.griffin.measure.rule.expr.{Expr, StatementExpr} +import org.apache.griffin.measure.rule.{ExprValueUtil, RuleAnalyzer, RuleFactory} import org.apache.spark.rdd.RDD import org.apache.spark.sql.hive.HiveContext import org.apache.spark.{SparkConf, SparkContext} diff --git a/measure/src/main/scala/org/apache/griffin/measure/batch/algo/core/AccuracyCore.scala b/measure/src/main/scala/org/apache/griffin/measure/algo/core/AccuracyCore.scala similarity index 95% rename from measure/src/main/scala/org/apache/griffin/measure/batch/algo/core/AccuracyCore.scala rename to measure/src/main/scala/org/apache/griffin/measure/algo/core/AccuracyCore.scala index f37c68e5a..cbea053b1 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/batch/algo/core/AccuracyCore.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/algo/core/AccuracyCore.scala @@ -12,10 +12,10 @@ See the License for the specific language governing permissions and limitations under the License. */ -package org.apache.griffin.measure.batch.algo.core +package org.apache.griffin.measure.algo.core -import org.apache.griffin.measure.batch.rule.RuleAnalyzer -import org.apache.griffin.measure.batch.result._ +import org.apache.griffin.measure.rule.RuleAnalyzer +import org.apache.griffin.measure.result._ import org.apache.spark.rdd.RDD diff --git a/measure/src/main/scala/org/apache/griffin/measure/batch/algo/core/ProfileCore.scala b/measure/src/main/scala/org/apache/griffin/measure/algo/core/ProfileCore.scala similarity index 93% rename from measure/src/main/scala/org/apache/griffin/measure/batch/algo/core/ProfileCore.scala rename to measure/src/main/scala/org/apache/griffin/measure/algo/core/ProfileCore.scala index dbe96b26d..dc59e336c 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/batch/algo/core/ProfileCore.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/algo/core/ProfileCore.scala @@ -12,10 +12,10 @@ See the License for the specific language governing permissions and limitations under the License. */ -package org.apache.griffin.measure.batch.algo.core +package org.apache.griffin.measure.algo.core -import org.apache.griffin.measure.batch.rule.RuleAnalyzer -import org.apache.griffin.measure.batch.result._ +import org.apache.griffin.measure.rule.RuleAnalyzer +import org.apache.griffin.measure.result._ import org.apache.spark.rdd.RDD diff --git a/measure/src/main/scala/org/apache/griffin/measure/batch/config/params/AllParam.scala b/measure/src/main/scala/org/apache/griffin/measure/config/params/AllParam.scala similarity index 84% rename from measure/src/main/scala/org/apache/griffin/measure/batch/config/params/AllParam.scala rename to measure/src/main/scala/org/apache/griffin/measure/config/params/AllParam.scala index 4e2d5d824..1e0f86a08 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/batch/config/params/AllParam.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/config/params/AllParam.scala @@ -12,12 +12,12 @@ See the License for the specific language governing permissions and limitations under the License. */ -package org.apache.griffin.measure.batch.config.params +package org.apache.griffin.measure.config.params import com.fasterxml.jackson.annotation.{JsonInclude, JsonProperty} import com.fasterxml.jackson.annotation.JsonInclude.Include -import org.apache.griffin.measure.batch.config.params.env._ -import org.apache.griffin.measure.batch.config.params.user._ +import org.apache.griffin.measure.config.params.env._ +import org.apache.griffin.measure.config.params.user._ // simply composite of env and user params, for convenient usage @JsonInclude(Include.NON_NULL) diff --git a/measure/src/main/scala/org/apache/griffin/measure/batch/config/params/Param.scala b/measure/src/main/scala/org/apache/griffin/measure/config/params/Param.scala similarity index 91% rename from measure/src/main/scala/org/apache/griffin/measure/batch/config/params/Param.scala rename to measure/src/main/scala/org/apache/griffin/measure/config/params/Param.scala index 437ac67b2..46ebdeb92 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/batch/config/params/Param.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/config/params/Param.scala @@ -12,7 +12,7 @@ See the License for the specific language governing permissions and limitations under the License. */ -package org.apache.griffin.measure.batch.config.params +package org.apache.griffin.measure.config.params trait Param extends Serializable { diff --git a/measure/src/main/scala/org/apache/griffin/measure/batch/config/params/env/CleanerParam.scala b/measure/src/main/scala/org/apache/griffin/measure/config/params/env/CleanerParam.scala similarity index 86% rename from measure/src/main/scala/org/apache/griffin/measure/batch/config/params/env/CleanerParam.scala rename to measure/src/main/scala/org/apache/griffin/measure/config/params/env/CleanerParam.scala index 48969322c..e5d2f093b 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/batch/config/params/env/CleanerParam.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/config/params/env/CleanerParam.scala @@ -12,11 +12,11 @@ See the License for the specific language governing permissions and limitations under the License. */ -package org.apache.griffin.measure.batch.config.params.env +package org.apache.griffin.measure.config.params.env import com.fasterxml.jackson.annotation.{JsonInclude, JsonProperty} import com.fasterxml.jackson.annotation.JsonInclude.Include -import org.apache.griffin.measure.batch.config.params.Param +import org.apache.griffin.measure.config.params.Param @JsonInclude(Include.NON_NULL) case class CleanerParam() extends Param { diff --git a/measure/src/main/scala/org/apache/griffin/measure/batch/config/params/env/EnvParam.scala b/measure/src/main/scala/org/apache/griffin/measure/config/params/env/EnvParam.scala similarity index 88% rename from measure/src/main/scala/org/apache/griffin/measure/batch/config/params/env/EnvParam.scala rename to measure/src/main/scala/org/apache/griffin/measure/config/params/env/EnvParam.scala index a2aa7a518..ccef5627b 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/batch/config/params/env/EnvParam.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/config/params/env/EnvParam.scala @@ -12,11 +12,11 @@ See the License for the specific language governing permissions and limitations under the License. */ -package org.apache.griffin.measure.batch.config.params.env +package org.apache.griffin.measure.config.params.env import com.fasterxml.jackson.annotation.{JsonInclude, JsonProperty} import com.fasterxml.jackson.annotation.JsonInclude.Include -import org.apache.griffin.measure.batch.config.params.Param +import org.apache.griffin.measure.config.params.Param @JsonInclude(Include.NON_NULL) case class EnvParam( @JsonProperty("spark") sparkParam: SparkParam, diff --git a/measure/src/main/scala/org/apache/griffin/measure/batch/config/params/env/PersistParam.scala b/measure/src/main/scala/org/apache/griffin/measure/config/params/env/PersistParam.scala similarity index 88% rename from measure/src/main/scala/org/apache/griffin/measure/batch/config/params/env/PersistParam.scala rename to measure/src/main/scala/org/apache/griffin/measure/config/params/env/PersistParam.scala index aaffda5ce..4aaa059d4 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/batch/config/params/env/PersistParam.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/config/params/env/PersistParam.scala @@ -12,11 +12,11 @@ See the License for the specific language governing permissions and limitations under the License. */ -package org.apache.griffin.measure.batch.config.params.env +package org.apache.griffin.measure.config.params.env import com.fasterxml.jackson.annotation.{JsonInclude, JsonProperty} import com.fasterxml.jackson.annotation.JsonInclude.Include -import org.apache.griffin.measure.batch.config.params.Param +import org.apache.griffin.measure.config.params.Param @JsonInclude(Include.NON_NULL) case class PersistParam( @JsonProperty("type") persistType: String, diff --git a/measure/src/main/scala/org/apache/griffin/measure/batch/config/params/env/SparkParam.scala b/measure/src/main/scala/org/apache/griffin/measure/config/params/env/SparkParam.scala similarity index 88% rename from measure/src/main/scala/org/apache/griffin/measure/batch/config/params/env/SparkParam.scala rename to measure/src/main/scala/org/apache/griffin/measure/config/params/env/SparkParam.scala index 9979f19f7..c291ca79a 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/batch/config/params/env/SparkParam.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/config/params/env/SparkParam.scala @@ -12,11 +12,11 @@ See the License for the specific language governing permissions and limitations under the License. */ -package org.apache.griffin.measure.batch.config.params.env +package org.apache.griffin.measure.config.params.env import com.fasterxml.jackson.annotation.{JsonInclude, JsonProperty} import com.fasterxml.jackson.annotation.JsonInclude.Include -import org.apache.griffin.measure.batch.config.params.Param +import org.apache.griffin.measure.config.params.Param @JsonInclude(Include.NON_NULL) case class SparkParam( @JsonProperty("log.level") logLevel: String, diff --git a/measure/src/main/scala/org/apache/griffin/measure/batch/config/params/user/DataConnectorParam.scala b/measure/src/main/scala/org/apache/griffin/measure/config/params/user/DataConnectorParam.scala similarity index 89% rename from measure/src/main/scala/org/apache/griffin/measure/batch/config/params/user/DataConnectorParam.scala rename to measure/src/main/scala/org/apache/griffin/measure/config/params/user/DataConnectorParam.scala index 3480c1f98..74e104c31 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/batch/config/params/user/DataConnectorParam.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/config/params/user/DataConnectorParam.scala @@ -12,11 +12,11 @@ See the License for the specific language governing permissions and limitations under the License. */ -package org.apache.griffin.measure.batch.config.params.user +package org.apache.griffin.measure.config.params.user import com.fasterxml.jackson.annotation.{JsonInclude, JsonProperty} import com.fasterxml.jackson.annotation.JsonInclude.Include -import org.apache.griffin.measure.batch.config.params.Param +import org.apache.griffin.measure.config.params.Param @JsonInclude(Include.NON_NULL) case class DataConnectorParam( @JsonProperty("type") conType: String, diff --git a/measure/src/main/scala/org/apache/griffin/measure/batch/config/params/user/EvaluateRuleParam.scala b/measure/src/main/scala/org/apache/griffin/measure/config/params/user/EvaluateRuleParam.scala similarity index 88% rename from measure/src/main/scala/org/apache/griffin/measure/batch/config/params/user/EvaluateRuleParam.scala rename to measure/src/main/scala/org/apache/griffin/measure/config/params/user/EvaluateRuleParam.scala index aff113b95..8db6ec21c 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/batch/config/params/user/EvaluateRuleParam.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/config/params/user/EvaluateRuleParam.scala @@ -12,11 +12,11 @@ See the License for the specific language governing permissions and limitations under the License. */ -package org.apache.griffin.measure.batch.config.params.user +package org.apache.griffin.measure.config.params.user import com.fasterxml.jackson.annotation.{JsonInclude, JsonProperty} import com.fasterxml.jackson.annotation.JsonInclude.Include -import org.apache.griffin.measure.batch.config.params.Param +import org.apache.griffin.measure.config.params.Param @JsonInclude(Include.NON_NULL) case class EvaluateRuleParam( @JsonProperty("sampleRatio") sampleRatio: Double, diff --git a/measure/src/main/scala/org/apache/griffin/measure/batch/config/params/user/UserParam.scala b/measure/src/main/scala/org/apache/griffin/measure/config/params/user/UserParam.scala similarity index 90% rename from measure/src/main/scala/org/apache/griffin/measure/batch/config/params/user/UserParam.scala rename to measure/src/main/scala/org/apache/griffin/measure/config/params/user/UserParam.scala index b11636169..24bd25eb4 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/batch/config/params/user/UserParam.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/config/params/user/UserParam.scala @@ -12,11 +12,11 @@ See the License for the specific language governing permissions and limitations under the License. */ -package org.apache.griffin.measure.batch.config.params.user +package org.apache.griffin.measure.config.params.user import com.fasterxml.jackson.annotation.{JsonInclude, JsonProperty} import com.fasterxml.jackson.annotation.JsonInclude.Include -import org.apache.griffin.measure.batch.config.params.Param +import org.apache.griffin.measure.config.params.Param @JsonInclude(Include.NON_NULL) case class UserParam(@JsonProperty("name") name: String, diff --git a/measure/src/main/scala/org/apache/griffin/measure/batch/config/reader/ParamFileReader.scala b/measure/src/main/scala/org/apache/griffin/measure/config/reader/ParamFileReader.scala similarity index 83% rename from measure/src/main/scala/org/apache/griffin/measure/batch/config/reader/ParamFileReader.scala rename to measure/src/main/scala/org/apache/griffin/measure/config/reader/ParamFileReader.scala index 94b3352f9..ce9d9f3be 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/batch/config/reader/ParamFileReader.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/config/reader/ParamFileReader.scala @@ -12,10 +12,10 @@ See the License for the specific language governing permissions and limitations under the License. */ -package org.apache.griffin.measure.batch.config.reader +package org.apache.griffin.measure.config.reader -import org.apache.griffin.measure.batch.config.params.Param -import org.apache.griffin.measure.batch.utils.JsonUtil +import org.apache.griffin.measure.config.params.Param +import org.apache.griffin.measure.utils.JsonUtil import scala.util.Try diff --git a/measure/src/main/scala/org/apache/griffin/measure/batch/config/reader/ParamHdfsFileReader.scala b/measure/src/main/scala/org/apache/griffin/measure/config/reader/ParamHdfsFileReader.scala similarity index 79% rename from measure/src/main/scala/org/apache/griffin/measure/batch/config/reader/ParamHdfsFileReader.scala rename to measure/src/main/scala/org/apache/griffin/measure/config/reader/ParamHdfsFileReader.scala index 2755e13b8..227545f3a 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/batch/config/reader/ParamHdfsFileReader.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/config/reader/ParamHdfsFileReader.scala @@ -12,11 +12,11 @@ See the License for the specific language governing permissions and limitations under the License. */ -package org.apache.griffin.measure.batch.config.reader +package org.apache.griffin.measure.config.reader -import org.apache.griffin.measure.batch.config.params.Param -import org.apache.griffin.measure.batch.utils.JsonUtil -import org.apache.griffin.measure.batch.utils.HdfsUtil +import org.apache.griffin.measure.config.params.Param +import org.apache.griffin.measure.utils.JsonUtil +import org.apache.griffin.measure.utils.HdfsUtil import scala.util.Try diff --git a/measure/src/main/scala/org/apache/griffin/measure/batch/config/reader/ParamRawStringReader.scala b/measure/src/main/scala/org/apache/griffin/measure/config/reader/ParamRawStringReader.scala similarity index 82% rename from measure/src/main/scala/org/apache/griffin/measure/batch/config/reader/ParamRawStringReader.scala rename to measure/src/main/scala/org/apache/griffin/measure/config/reader/ParamRawStringReader.scala index 0aef6994f..63472985b 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/batch/config/reader/ParamRawStringReader.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/config/reader/ParamRawStringReader.scala @@ -12,10 +12,10 @@ See the License for the specific language governing permissions and limitations under the License. */ -package org.apache.griffin.measure.batch.config.reader +package org.apache.griffin.measure.config.reader -import org.apache.griffin.measure.batch.config.params.Param -import org.apache.griffin.measure.batch.utils.JsonUtil +import org.apache.griffin.measure.config.params.Param +import org.apache.griffin.measure.utils.JsonUtil import scala.util.Try diff --git a/measure/src/main/scala/org/apache/griffin/measure/batch/config/reader/ParamReader.scala b/measure/src/main/scala/org/apache/griffin/measure/config/reader/ParamReader.scala similarity index 80% rename from measure/src/main/scala/org/apache/griffin/measure/batch/config/reader/ParamReader.scala rename to measure/src/main/scala/org/apache/griffin/measure/config/reader/ParamReader.scala index 9e9366d1a..219dbe1fd 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/batch/config/reader/ParamReader.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/config/reader/ParamReader.scala @@ -12,10 +12,10 @@ See the License for the specific language governing permissions and limitations under the License. */ -package org.apache.griffin.measure.batch.config.reader +package org.apache.griffin.measure.config.reader -import org.apache.griffin.measure.batch.log.Loggable -import org.apache.griffin.measure.batch.config.params.Param +import org.apache.griffin.measure.log.Loggable +import org.apache.griffin.measure.config.params.Param import scala.util.Try diff --git a/measure/src/main/scala/org/apache/griffin/measure/batch/config/reader/ParamReaderFactory.scala b/measure/src/main/scala/org/apache/griffin/measure/config/reader/ParamReaderFactory.scala similarity index 95% rename from measure/src/main/scala/org/apache/griffin/measure/batch/config/reader/ParamReaderFactory.scala rename to measure/src/main/scala/org/apache/griffin/measure/config/reader/ParamReaderFactory.scala index bfa6a68e9..5dbbf0cab 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/batch/config/reader/ParamReaderFactory.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/config/reader/ParamReaderFactory.scala @@ -12,7 +12,7 @@ See the License for the specific language governing permissions and limitations under the License. */ -package org.apache.griffin.measure.batch.config.reader +package org.apache.griffin.measure.config.reader import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileSystem, Path} diff --git a/measure/src/main/scala/org/apache/griffin/measure/batch/config/validator/AllParamValidator.scala b/measure/src/main/scala/org/apache/griffin/measure/config/validator/AllParamValidator.scala similarity index 86% rename from measure/src/main/scala/org/apache/griffin/measure/batch/config/validator/AllParamValidator.scala rename to measure/src/main/scala/org/apache/griffin/measure/config/validator/AllParamValidator.scala index a89d9f16e..afff15d6d 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/batch/config/validator/AllParamValidator.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/config/validator/AllParamValidator.scala @@ -12,9 +12,9 @@ See the License for the specific language governing permissions and limitations under the License. */ -package org.apache.griffin.measure.batch.config.validator +package org.apache.griffin.measure.config.validator -import org.apache.griffin.measure.batch.config.params.Param +import org.apache.griffin.measure.config.params.Param import scala.util.Try diff --git a/measure/src/main/scala/org/apache/griffin/measure/batch/config/validator/ParamValidator.scala b/measure/src/main/scala/org/apache/griffin/measure/config/validator/ParamValidator.scala similarity index 79% rename from measure/src/main/scala/org/apache/griffin/measure/batch/config/validator/ParamValidator.scala rename to measure/src/main/scala/org/apache/griffin/measure/config/validator/ParamValidator.scala index 8a78f08a1..c07016f65 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/batch/config/validator/ParamValidator.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/config/validator/ParamValidator.scala @@ -12,10 +12,10 @@ See the License for the specific language governing permissions and limitations under the License. */ -package org.apache.griffin.measure.batch.config.validator +package org.apache.griffin.measure.config.validator -import org.apache.griffin.measure.batch.log.Loggable -import org.apache.griffin.measure.batch.config.params.Param +import org.apache.griffin.measure.log.Loggable +import org.apache.griffin.measure.config.params.Param import scala.util.Try diff --git a/measure/src/main/scala/org/apache/griffin/measure/batch/connector/AvroDataConnector.scala b/measure/src/main/scala/org/apache/griffin/measure/connector/AvroDataConnector.scala similarity index 93% rename from measure/src/main/scala/org/apache/griffin/measure/batch/connector/AvroDataConnector.scala rename to measure/src/main/scala/org/apache/griffin/measure/connector/AvroDataConnector.scala index 3835c9d92..04ed07d34 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/batch/connector/AvroDataConnector.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/connector/AvroDataConnector.scala @@ -12,9 +12,9 @@ See the License for the specific language governing permissions and limitations under the License. */ -package org.apache.griffin.measure.batch.connector +package org.apache.griffin.measure.connector -import org.apache.griffin.measure.batch.rule.expr._ +import org.apache.griffin.measure.rule.expr._ import org.apache.spark.rdd.RDD import org.apache.spark.sql.{DataFrame, Row, SQLContext} import com.databricks.spark.avro._ @@ -22,8 +22,8 @@ import com.databricks.spark.avro._ import scala.util.{Success, Try} import java.nio.file.{Files, Paths} -import org.apache.griffin.measure.batch.rule.{ExprValueUtil, RuleExprs} -import org.apache.griffin.measure.batch.utils.HdfsUtil +import org.apache.griffin.measure.rule.{ExprValueUtil, RuleExprs} +import org.apache.griffin.measure.utils.HdfsUtil // data connector for avro file case class AvroDataConnector(sqlContext: SQLContext, config: Map[String, Any], diff --git a/measure/src/main/scala/org/apache/griffin/measure/batch/connector/DataConnector.scala b/measure/src/main/scala/org/apache/griffin/measure/connector/DataConnector.scala similarity index 93% rename from measure/src/main/scala/org/apache/griffin/measure/batch/connector/DataConnector.scala rename to measure/src/main/scala/org/apache/griffin/measure/connector/DataConnector.scala index 7c7cb8e80..0189cdaed 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/batch/connector/DataConnector.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/connector/DataConnector.scala @@ -12,7 +12,7 @@ See the License for the specific language governing permissions and limitations under the License. */ -package org.apache.griffin.measure.batch.connector +package org.apache.griffin.measure.connector import org.apache.spark.rdd.RDD diff --git a/measure/src/main/scala/org/apache/griffin/measure/batch/connector/DataConnectorFactory.scala b/measure/src/main/scala/org/apache/griffin/measure/connector/DataConnectorFactory.scala similarity index 86% rename from measure/src/main/scala/org/apache/griffin/measure/batch/connector/DataConnectorFactory.scala rename to measure/src/main/scala/org/apache/griffin/measure/connector/DataConnectorFactory.scala index b58697417..583e86d6a 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/batch/connector/DataConnectorFactory.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/connector/DataConnectorFactory.scala @@ -12,11 +12,11 @@ See the License for the specific language governing permissions and limitations under the License. */ -package org.apache.griffin.measure.batch.connector +package org.apache.griffin.measure.connector -import org.apache.griffin.measure.batch.config.params.user._ -import org.apache.griffin.measure.batch.rule.RuleExprs -import org.apache.griffin.measure.batch.rule.expr._ +import org.apache.griffin.measure.config.params.user._ +import org.apache.griffin.measure.rule.RuleExprs +import org.apache.griffin.measure.rule.expr._ import org.apache.spark.sql.SQLContext import scala.util.Try diff --git a/measure/src/main/scala/org/apache/griffin/measure/batch/connector/HiveDataConnector.scala b/measure/src/main/scala/org/apache/griffin/measure/connector/HiveDataConnector.scala similarity index 95% rename from measure/src/main/scala/org/apache/griffin/measure/batch/connector/HiveDataConnector.scala rename to measure/src/main/scala/org/apache/griffin/measure/connector/HiveDataConnector.scala index de4cffb20..41bd073d0 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/batch/connector/HiveDataConnector.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/connector/HiveDataConnector.scala @@ -12,10 +12,10 @@ See the License for the specific language governing permissions and limitations under the License. */ -package org.apache.griffin.measure.batch.connector +package org.apache.griffin.measure.connector -import org.apache.griffin.measure.batch.rule.{ExprValueUtil, RuleExprs} -import org.apache.griffin.measure.batch.rule.expr._ +import org.apache.griffin.measure.rule.{ExprValueUtil, RuleExprs} +import org.apache.griffin.measure.rule.expr._ import org.apache.spark.rdd.RDD import org.apache.spark.sql.{DataFrame, Row, SQLContext} diff --git a/measure/src/main/scala/org/apache/griffin/measure/batch/log/Loggable.scala b/measure/src/main/scala/org/apache/griffin/measure/log/Loggable.scala similarity index 95% rename from measure/src/main/scala/org/apache/griffin/measure/batch/log/Loggable.scala rename to measure/src/main/scala/org/apache/griffin/measure/log/Loggable.scala index 02f1a994e..bd4f13b47 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/batch/log/Loggable.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/log/Loggable.scala @@ -12,7 +12,7 @@ See the License for the specific language governing permissions and limitations under the License. */ -package org.apache.griffin.measure.batch.log +package org.apache.griffin.measure.log import org.slf4j.LoggerFactory diff --git a/measure/src/main/scala/org/apache/griffin/measure/batch/persist/HdfsPersist.scala b/measure/src/main/scala/org/apache/griffin/measure/persist/HdfsPersist.scala similarity index 96% rename from measure/src/main/scala/org/apache/griffin/measure/batch/persist/HdfsPersist.scala rename to measure/src/main/scala/org/apache/griffin/measure/persist/HdfsPersist.scala index 7a6a94b66..67b86f135 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/batch/persist/HdfsPersist.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/persist/HdfsPersist.scala @@ -12,12 +12,12 @@ See the License for the specific language governing permissions and limitations under the License. */ -package org.apache.griffin.measure.batch.persist +package org.apache.griffin.measure.persist import java.util.Date -import org.apache.griffin.measure.batch.result._ -import org.apache.griffin.measure.batch.utils.HdfsUtil +import org.apache.griffin.measure.result._ +import org.apache.griffin.measure.utils.HdfsUtil import org.apache.spark.rdd.RDD import scala.util.Try diff --git a/measure/src/main/scala/org/apache/griffin/measure/batch/persist/HttpPersist.scala b/measure/src/main/scala/org/apache/griffin/measure/persist/HttpPersist.scala similarity index 92% rename from measure/src/main/scala/org/apache/griffin/measure/batch/persist/HttpPersist.scala rename to measure/src/main/scala/org/apache/griffin/measure/persist/HttpPersist.scala index 32232489a..8c0438613 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/batch/persist/HttpPersist.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/persist/HttpPersist.scala @@ -12,10 +12,10 @@ See the License for the specific language governing permissions and limitations under the License. */ -package org.apache.griffin.measure.batch.persist +package org.apache.griffin.measure.persist -import org.apache.griffin.measure.batch.result._ -import org.apache.griffin.measure.batch.utils.{HttpUtil, JsonUtil} +import org.apache.griffin.measure.result._ +import org.apache.griffin.measure.utils.{HttpUtil, JsonUtil} import org.apache.spark.rdd.RDD import scala.util.Try diff --git a/measure/src/main/scala/org/apache/griffin/measure/batch/persist/LoggerPersist.scala b/measure/src/main/scala/org/apache/griffin/measure/persist/LoggerPersist.scala similarity index 94% rename from measure/src/main/scala/org/apache/griffin/measure/batch/persist/LoggerPersist.scala rename to measure/src/main/scala/org/apache/griffin/measure/persist/LoggerPersist.scala index da909c0b4..db45f4faa 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/batch/persist/LoggerPersist.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/persist/LoggerPersist.scala @@ -12,12 +12,12 @@ See the License for the specific language governing permissions and limitations under the License. */ -package org.apache.griffin.measure.batch.persist +package org.apache.griffin.measure.persist import java.util.Date -import org.apache.griffin.measure.batch.result._ -import org.apache.griffin.measure.batch.utils.HdfsUtil +import org.apache.griffin.measure.result._ +import org.apache.griffin.measure.utils.HdfsUtil import org.apache.spark.rdd.RDD // persist result and data to hdfs diff --git a/measure/src/main/scala/org/apache/griffin/measure/batch/persist/MultiPersists.scala b/measure/src/main/scala/org/apache/griffin/measure/persist/MultiPersists.scala similarity index 89% rename from measure/src/main/scala/org/apache/griffin/measure/batch/persist/MultiPersists.scala rename to measure/src/main/scala/org/apache/griffin/measure/persist/MultiPersists.scala index ce07eca98..5674729f9 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/batch/persist/MultiPersists.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/persist/MultiPersists.scala @@ -12,10 +12,10 @@ See the License for the specific language governing permissions and limitations under the License. */ -package org.apache.griffin.measure.batch.persist +package org.apache.griffin.measure.persist -import org.apache.griffin.measure.batch.result._ -import org.apache.griffin.measure.batch.utils.{HttpUtil, JsonUtil} +import org.apache.griffin.measure.result._ +import org.apache.griffin.measure.utils.{HttpUtil, JsonUtil} import org.apache.spark.rdd.RDD import scala.util.Try diff --git a/measure/src/main/scala/org/apache/griffin/measure/batch/persist/Persist.scala b/measure/src/main/scala/org/apache/griffin/measure/persist/Persist.scala similarity index 86% rename from measure/src/main/scala/org/apache/griffin/measure/batch/persist/Persist.scala rename to measure/src/main/scala/org/apache/griffin/measure/persist/Persist.scala index 77ee61ff3..0ecfa213f 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/batch/persist/Persist.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/persist/Persist.scala @@ -12,10 +12,10 @@ See the License for the specific language governing permissions and limitations under the License. */ -package org.apache.griffin.measure.batch.persist +package org.apache.griffin.measure.persist -import org.apache.griffin.measure.batch.log.Loggable -import org.apache.griffin.measure.batch.result._ +import org.apache.griffin.measure.log.Loggable +import org.apache.griffin.measure.result._ import org.apache.spark.rdd.RDD import scala.util.Try diff --git a/measure/src/main/scala/org/apache/griffin/measure/batch/persist/PersistFactory.scala b/measure/src/main/scala/org/apache/griffin/measure/persist/PersistFactory.scala similarity index 93% rename from measure/src/main/scala/org/apache/griffin/measure/batch/persist/PersistFactory.scala rename to measure/src/main/scala/org/apache/griffin/measure/persist/PersistFactory.scala index c18db9223..cae6730d6 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/batch/persist/PersistFactory.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/persist/PersistFactory.scala @@ -12,9 +12,9 @@ See the License for the specific language governing permissions and limitations under the License. */ -package org.apache.griffin.measure.batch.persist +package org.apache.griffin.measure.persist -import org.apache.griffin.measure.batch.config.params.env._ +import org.apache.griffin.measure.config.params.env._ import scala.util.{Success, Try} diff --git a/measure/src/main/scala/org/apache/griffin/measure/batch/result/AccuracyResult.scala b/measure/src/main/scala/org/apache/griffin/measure/result/AccuracyResult.scala similarity index 95% rename from measure/src/main/scala/org/apache/griffin/measure/batch/result/AccuracyResult.scala rename to measure/src/main/scala/org/apache/griffin/measure/result/AccuracyResult.scala index 3ccb14046..0ffab6670 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/batch/result/AccuracyResult.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/result/AccuracyResult.scala @@ -12,7 +12,7 @@ See the License for the specific language governing permissions and limitations under the License. */ -package org.apache.griffin.measure.batch.result +package org.apache.griffin.measure.result // result for accuracy: miss count, total count case class AccuracyResult(miss: Long, total: Long) extends Result { diff --git a/measure/src/main/scala/org/apache/griffin/measure/batch/result/ProfileResult.scala b/measure/src/main/scala/org/apache/griffin/measure/result/ProfileResult.scala similarity index 96% rename from measure/src/main/scala/org/apache/griffin/measure/batch/result/ProfileResult.scala rename to measure/src/main/scala/org/apache/griffin/measure/result/ProfileResult.scala index 41a06390e..0415c1f99 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/batch/result/ProfileResult.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/result/ProfileResult.scala @@ -12,7 +12,7 @@ See the License for the specific language governing permissions and limitations under the License. */ -package org.apache.griffin.measure.batch.result +package org.apache.griffin.measure.result // result for profile: match count, total count case class ProfileResult(matchCount: Long, totalCount: Long) extends Result { diff --git a/measure/src/main/scala/org/apache/griffin/measure/batch/result/Result.scala b/measure/src/main/scala/org/apache/griffin/measure/result/Result.scala similarity index 93% rename from measure/src/main/scala/org/apache/griffin/measure/batch/result/Result.scala rename to measure/src/main/scala/org/apache/griffin/measure/result/Result.scala index d0df265ce..5ac8935b7 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/batch/result/Result.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/result/Result.scala @@ -12,7 +12,7 @@ See the License for the specific language governing permissions and limitations under the License. */ -package org.apache.griffin.measure.batch.result +package org.apache.griffin.measure.result trait Result extends Serializable { diff --git a/measure/src/main/scala/org/apache/griffin/measure/batch/result/ResultInfo.scala b/measure/src/main/scala/org/apache/griffin/measure/result/ResultInfo.scala similarity index 96% rename from measure/src/main/scala/org/apache/griffin/measure/batch/result/ResultInfo.scala rename to measure/src/main/scala/org/apache/griffin/measure/result/ResultInfo.scala index 731190da2..2020caf64 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/batch/result/ResultInfo.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/result/ResultInfo.scala @@ -12,7 +12,7 @@ See the License for the specific language governing permissions and limitations under the License. */ -package org.apache.griffin.measure.batch.result +package org.apache.griffin.measure.result sealed trait ResultInfo { diff --git a/measure/src/main/scala/org/apache/griffin/measure/batch/rule/CalculationUtil.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/CalculationUtil.scala similarity index 99% rename from measure/src/main/scala/org/apache/griffin/measure/batch/rule/CalculationUtil.scala rename to measure/src/main/scala/org/apache/griffin/measure/rule/CalculationUtil.scala index 0c26a8b21..797d3ebef 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/batch/rule/CalculationUtil.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rule/CalculationUtil.scala @@ -12,7 +12,7 @@ See the License for the specific language governing permissions and limitations under the License. */ -package org.apache.griffin.measure.batch.rule +package org.apache.griffin.measure.rule import scala.util.{Success, Try} diff --git a/measure/src/main/scala/org/apache/griffin/measure/batch/rule/ExprValueUtil.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/ExprValueUtil.scala similarity index 97% rename from measure/src/main/scala/org/apache/griffin/measure/batch/rule/ExprValueUtil.scala rename to measure/src/main/scala/org/apache/griffin/measure/rule/ExprValueUtil.scala index d3c1b5bb2..8d97ee4d7 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/batch/rule/ExprValueUtil.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rule/ExprValueUtil.scala @@ -12,9 +12,9 @@ See the License for the specific language governing permissions and limitations under the License. */ -package org.apache.griffin.measure.batch.rule +package org.apache.griffin.measure.rule -import org.apache.griffin.measure.batch.rule.expr._ +import org.apache.griffin.measure.rule.expr._ import org.apache.spark.sql.Row import scala.util.{Success, Try} diff --git a/measure/src/main/scala/org/apache/griffin/measure/batch/rule/RuleAnalyzer.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/RuleAnalyzer.scala similarity index 97% rename from measure/src/main/scala/org/apache/griffin/measure/batch/rule/RuleAnalyzer.scala rename to measure/src/main/scala/org/apache/griffin/measure/rule/RuleAnalyzer.scala index a4b478f90..f791de302 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/batch/rule/RuleAnalyzer.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rule/RuleAnalyzer.scala @@ -12,9 +12,9 @@ See the License for the specific language governing permissions and limitations under the License. */ -package org.apache.griffin.measure.batch.rule +package org.apache.griffin.measure.rule -import org.apache.griffin.measure.batch.rule.expr._ +import org.apache.griffin.measure.rule.expr._ case class RuleAnalyzer(rule: StatementExpr) extends Serializable { diff --git a/measure/src/main/scala/org/apache/griffin/measure/batch/rule/RuleFactory.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/RuleFactory.scala similarity index 84% rename from measure/src/main/scala/org/apache/griffin/measure/batch/rule/RuleFactory.scala rename to measure/src/main/scala/org/apache/griffin/measure/rule/RuleFactory.scala index 7ba7ca47d..06d4a7b44 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/batch/rule/RuleFactory.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rule/RuleFactory.scala @@ -12,13 +12,13 @@ See the License for the specific language governing permissions and limitations under the License. */ -package org.apache.griffin.measure.batch.rule +package org.apache.griffin.measure.rule -import org.apache.griffin.measure.batch.config.params.user._ +import org.apache.griffin.measure.config.params.user._ import scala.util.Failure -//import org.apache.griffin.measure.batch.rule.expr_old._ -import org.apache.griffin.measure.batch.rule.expr._ +//import org.apache.griffin.measure.rule.expr_old._ +import org.apache.griffin.measure.rule.expr._ import scala.util.{Success, Try} diff --git a/measure/src/main/scala/org/apache/griffin/measure/batch/rule/RuleParser.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/RuleParser.scala similarity index 99% rename from measure/src/main/scala/org/apache/griffin/measure/batch/rule/RuleParser.scala rename to measure/src/main/scala/org/apache/griffin/measure/rule/RuleParser.scala index 29558dbab..aa419cee1 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/batch/rule/RuleParser.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rule/RuleParser.scala @@ -12,9 +12,9 @@ See the License for the specific language governing permissions and limitations under the License. */ -package org.apache.griffin.measure.batch.rule +package org.apache.griffin.measure.rule -import org.apache.griffin.measure.batch.rule.expr._ +import org.apache.griffin.measure.rule.expr._ import scala.util.parsing.combinator._ diff --git a/measure/src/main/scala/org/apache/griffin/measure/batch/rule/expr/AnalyzableExpr.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/expr/AnalyzableExpr.scala similarity index 93% rename from measure/src/main/scala/org/apache/griffin/measure/batch/rule/expr/AnalyzableExpr.scala rename to measure/src/main/scala/org/apache/griffin/measure/rule/expr/AnalyzableExpr.scala index d82450601..c9bc58e22 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/batch/rule/expr/AnalyzableExpr.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rule/expr/AnalyzableExpr.scala @@ -12,7 +12,7 @@ See the License for the specific language governing permissions and limitations under the License. */ -package org.apache.griffin.measure.batch.rule.expr +package org.apache.griffin.measure.rule.expr trait AnalyzableExpr extends Serializable { diff --git a/measure/src/main/scala/org/apache/griffin/measure/batch/rule/expr/Cacheable.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/expr/Cacheable.scala similarity index 95% rename from measure/src/main/scala/org/apache/griffin/measure/batch/rule/expr/Cacheable.scala rename to measure/src/main/scala/org/apache/griffin/measure/rule/expr/Cacheable.scala index 776a22dca..1d94f0755 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/batch/rule/expr/Cacheable.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rule/expr/Cacheable.scala @@ -12,7 +12,7 @@ See the License for the specific language governing permissions and limitations under the License. */ -package org.apache.griffin.measure.batch.rule.expr +package org.apache.griffin.measure.rule.expr trait Cacheable extends DataSourceable { protected def cacheUnit: Boolean = false diff --git a/measure/src/main/scala/org/apache/griffin/measure/batch/rule/expr/Calculatable.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/expr/Calculatable.scala similarity index 92% rename from measure/src/main/scala/org/apache/griffin/measure/batch/rule/expr/Calculatable.scala rename to measure/src/main/scala/org/apache/griffin/measure/rule/expr/Calculatable.scala index 1bbb1230e..860e79ee6 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/batch/rule/expr/Calculatable.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rule/expr/Calculatable.scala @@ -12,7 +12,7 @@ See the License for the specific language governing permissions and limitations under the License. */ -package org.apache.griffin.measure.batch.rule.expr +package org.apache.griffin.measure.rule.expr trait Calculatable extends Serializable { diff --git a/measure/src/main/scala/org/apache/griffin/measure/batch/rule/expr/DataSourceable.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/expr/DataSourceable.scala similarity index 94% rename from measure/src/main/scala/org/apache/griffin/measure/batch/rule/expr/DataSourceable.scala rename to measure/src/main/scala/org/apache/griffin/measure/rule/expr/DataSourceable.scala index c517e13c3..67bd2615f 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/batch/rule/expr/DataSourceable.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rule/expr/DataSourceable.scala @@ -12,7 +12,7 @@ See the License for the specific language governing permissions and limitations under the License. */ -package org.apache.griffin.measure.batch.rule.expr +package org.apache.griffin.measure.rule.expr trait DataSourceable extends Serializable { val dataSources: Set[String] diff --git a/measure/src/main/scala/org/apache/griffin/measure/batch/rule/expr/Describable.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/expr/Describable.scala similarity index 93% rename from measure/src/main/scala/org/apache/griffin/measure/batch/rule/expr/Describable.scala rename to measure/src/main/scala/org/apache/griffin/measure/rule/expr/Describable.scala index 5c91ecd7d..0e6384fbb 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/batch/rule/expr/Describable.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rule/expr/Describable.scala @@ -12,7 +12,7 @@ See the License for the specific language governing permissions and limitations under the License. */ -package org.apache.griffin.measure.batch.rule.expr +package org.apache.griffin.measure.rule.expr trait Describable extends Serializable { diff --git a/measure/src/main/scala/org/apache/griffin/measure/batch/rule/expr/Expr.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/expr/Expr.scala similarity index 96% rename from measure/src/main/scala/org/apache/griffin/measure/batch/rule/expr/Expr.scala rename to measure/src/main/scala/org/apache/griffin/measure/rule/expr/Expr.scala index 140c6d757..74ec4573f 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/batch/rule/expr/Expr.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rule/expr/Expr.scala @@ -12,7 +12,7 @@ See the License for the specific language governing permissions and limitations under the License. */ -package org.apache.griffin.measure.batch.rule.expr +package org.apache.griffin.measure.rule.expr trait Expr extends Serializable with Describable with Cacheable with Calculatable { diff --git a/measure/src/main/scala/org/apache/griffin/measure/batch/rule/expr/ExprDescOnly.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/expr/ExprDescOnly.scala similarity index 95% rename from measure/src/main/scala/org/apache/griffin/measure/batch/rule/expr/ExprDescOnly.scala rename to measure/src/main/scala/org/apache/griffin/measure/rule/expr/ExprDescOnly.scala index b830bb910..17dab70c8 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/batch/rule/expr/ExprDescOnly.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rule/expr/ExprDescOnly.scala @@ -12,7 +12,7 @@ See the License for the specific language governing permissions and limitations under the License. */ -package org.apache.griffin.measure.batch.rule.expr +package org.apache.griffin.measure.rule.expr trait ExprDescOnly extends Describable { diff --git a/measure/src/main/scala/org/apache/griffin/measure/batch/rule/expr/ExprIdCounter.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/expr/ExprIdCounter.scala similarity index 96% rename from measure/src/main/scala/org/apache/griffin/measure/batch/rule/expr/ExprIdCounter.scala rename to measure/src/main/scala/org/apache/griffin/measure/rule/expr/ExprIdCounter.scala index 2eb4475d5..790d6fede 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/batch/rule/expr/ExprIdCounter.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rule/expr/ExprIdCounter.scala @@ -12,7 +12,7 @@ See the License for the specific language governing permissions and limitations under the License. */ -package org.apache.griffin.measure.batch.rule.expr +package org.apache.griffin.measure.rule.expr import java.util.concurrent.atomic.AtomicLong diff --git a/measure/src/main/scala/org/apache/griffin/measure/batch/rule/expr/FieldDescOnly.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/expr/FieldDescOnly.scala similarity index 97% rename from measure/src/main/scala/org/apache/griffin/measure/batch/rule/expr/FieldDescOnly.scala rename to measure/src/main/scala/org/apache/griffin/measure/rule/expr/FieldDescOnly.scala index d28f54f40..3c10cfdd2 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/batch/rule/expr/FieldDescOnly.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rule/expr/FieldDescOnly.scala @@ -12,7 +12,7 @@ See the License for the specific language governing permissions and limitations under the License. */ -package org.apache.griffin.measure.batch.rule.expr +package org.apache.griffin.measure.rule.expr import scala.util.{Success, Try} diff --git a/measure/src/main/scala/org/apache/griffin/measure/batch/rule/expr/LiteralExpr.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/expr/LiteralExpr.scala similarity index 98% rename from measure/src/main/scala/org/apache/griffin/measure/batch/rule/expr/LiteralExpr.scala rename to measure/src/main/scala/org/apache/griffin/measure/rule/expr/LiteralExpr.scala index 2d004dabc..1a4dcb57c 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/batch/rule/expr/LiteralExpr.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rule/expr/LiteralExpr.scala @@ -12,7 +12,7 @@ See the License for the specific language governing permissions and limitations under the License. */ -package org.apache.griffin.measure.batch.rule.expr +package org.apache.griffin.measure.rule.expr import scala.util.{Failure, Success, Try} diff --git a/measure/src/main/scala/org/apache/griffin/measure/batch/rule/expr/LogicalExpr.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/expr/LogicalExpr.scala similarity index 98% rename from measure/src/main/scala/org/apache/griffin/measure/batch/rule/expr/LogicalExpr.scala rename to measure/src/main/scala/org/apache/griffin/measure/rule/expr/LogicalExpr.scala index d6ce357d2..95cf19236 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/batch/rule/expr/LogicalExpr.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rule/expr/LogicalExpr.scala @@ -12,9 +12,9 @@ See the License for the specific language governing permissions and limitations under the License. */ -package org.apache.griffin.measure.batch.rule.expr +package org.apache.griffin.measure.rule.expr -import org.apache.griffin.measure.batch.rule.CalculationUtil._ +import org.apache.griffin.measure.rule.CalculationUtil._ trait LogicalExpr extends Expr with AnalyzableExpr { override def cacheUnit: Boolean = true diff --git a/measure/src/main/scala/org/apache/griffin/measure/batch/rule/expr/MathExpr.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/expr/MathExpr.scala similarity index 96% rename from measure/src/main/scala/org/apache/griffin/measure/batch/rule/expr/MathExpr.scala rename to measure/src/main/scala/org/apache/griffin/measure/rule/expr/MathExpr.scala index 3d584dc31..ec8645ae0 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/batch/rule/expr/MathExpr.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rule/expr/MathExpr.scala @@ -12,9 +12,9 @@ See the License for the specific language governing permissions and limitations under the License. */ -package org.apache.griffin.measure.batch.rule.expr +package org.apache.griffin.measure.rule.expr -import org.apache.griffin.measure.batch.rule.CalculationUtil._ +import org.apache.griffin.measure.rule.CalculationUtil._ trait MathExpr extends Expr { diff --git a/measure/src/main/scala/org/apache/griffin/measure/batch/rule/expr/SelectExpr.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/expr/SelectExpr.scala similarity index 98% rename from measure/src/main/scala/org/apache/griffin/measure/batch/rule/expr/SelectExpr.scala rename to measure/src/main/scala/org/apache/griffin/measure/rule/expr/SelectExpr.scala index a25c9e276..04bad4247 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/batch/rule/expr/SelectExpr.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rule/expr/SelectExpr.scala @@ -12,7 +12,7 @@ See the License for the specific language governing permissions and limitations under the License. */ -package org.apache.griffin.measure.batch.rule.expr +package org.apache.griffin.measure.rule.expr trait SelectExpr extends Expr { def calculateOnly(values: Map[String, Any]): Option[Any] = None diff --git a/measure/src/main/scala/org/apache/griffin/measure/batch/rule/expr/StatementExpr.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/expr/StatementExpr.scala similarity index 97% rename from measure/src/main/scala/org/apache/griffin/measure/batch/rule/expr/StatementExpr.scala rename to measure/src/main/scala/org/apache/griffin/measure/rule/expr/StatementExpr.scala index 868d3bf11..7c4d18df0 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/batch/rule/expr/StatementExpr.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rule/expr/StatementExpr.scala @@ -12,7 +12,7 @@ See the License for the specific language governing permissions and limitations under the License. */ -package org.apache.griffin.measure.batch.rule.expr +package org.apache.griffin.measure.rule.expr trait StatementExpr extends Expr with AnalyzableExpr { diff --git a/measure/src/main/scala/org/apache/griffin/measure/batch/utils/HdfsUtil.scala b/measure/src/main/scala/org/apache/griffin/measure/utils/HdfsUtil.scala similarity index 97% rename from measure/src/main/scala/org/apache/griffin/measure/batch/utils/HdfsUtil.scala rename to measure/src/main/scala/org/apache/griffin/measure/utils/HdfsUtil.scala index 27531e8f9..79b0742c9 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/batch/utils/HdfsUtil.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/utils/HdfsUtil.scala @@ -12,7 +12,7 @@ See the License for the specific language governing permissions and limitations under the License. */ -package org.apache.griffin.measure.batch.utils +package org.apache.griffin.measure.utils import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FSDataInputStream, FSDataOutputStream, FileSystem, Path} diff --git a/measure/src/main/scala/org/apache/griffin/measure/batch/utils/HttpUtil.scala b/measure/src/main/scala/org/apache/griffin/measure/utils/HttpUtil.scala similarity index 97% rename from measure/src/main/scala/org/apache/griffin/measure/batch/utils/HttpUtil.scala rename to measure/src/main/scala/org/apache/griffin/measure/utils/HttpUtil.scala index e0be422fb..be754a02a 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/batch/utils/HttpUtil.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/utils/HttpUtil.scala @@ -12,7 +12,7 @@ See the License for the specific language governing permissions and limitations under the License. */ -package org.apache.griffin.measure.batch.utils +package org.apache.griffin.measure.utils import scalaj.http._ diff --git a/measure/src/main/scala/org/apache/griffin/measure/batch/utils/JsonUtil.scala b/measure/src/main/scala/org/apache/griffin/measure/utils/JsonUtil.scala similarity index 96% rename from measure/src/main/scala/org/apache/griffin/measure/batch/utils/JsonUtil.scala rename to measure/src/main/scala/org/apache/griffin/measure/utils/JsonUtil.scala index d3a069772..aee13d0fb 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/batch/utils/JsonUtil.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/utils/JsonUtil.scala @@ -12,7 +12,7 @@ See the License for the specific language governing permissions and limitations under the License. */ -package org.apache.griffin.measure.batch.utils +package org.apache.griffin.measure.utils import java.io.InputStream diff --git a/measure/src/test/scala/org/apache/griffin/measure/batch/algo/BatchAccuracyAlgoTest.scala b/measure/src/test/scala/org/apache/griffin/measure/algo/BatchAccuracyAlgoTest.scala similarity index 91% rename from measure/src/test/scala/org/apache/griffin/measure/batch/algo/BatchAccuracyAlgoTest.scala rename to measure/src/test/scala/org/apache/griffin/measure/algo/BatchAccuracyAlgoTest.scala index b611f3c23..a62826115 100644 --- a/measure/src/test/scala/org/apache/griffin/measure/batch/algo/BatchAccuracyAlgoTest.scala +++ b/measure/src/test/scala/org/apache/griffin/measure/algo/BatchAccuracyAlgoTest.scala @@ -12,19 +12,19 @@ //limitations under the License. // // */ -// package org.apache.griffin.measure.batch.algo +// package org.apache.griffin.measure.algo // //import java.util.Date // -//import org.apache.griffin.measure.batch.config.params._ -//import org.apache.griffin.measure.batch.config.params.env._ -//import org.apache.griffin.measure.batch.config.params.user._ -//import org.apache.griffin.measure.batch.config.reader._ -//import org.apache.griffin.measure.batch.config.validator._ -//import org.apache.griffin.measure.batch.connector.{DataConnector, DataConnectorFactory} -//import org.apache.griffin.measure.batch.log.Loggable -//import org.apache.griffin.measure.batch.rule.expr._ -//import org.apache.griffin.measure.batch.rule.{ExprValueUtil, RuleAnalyzer, RuleFactory} +//import org.apache.griffin.measure.config.params._ +//import org.apache.griffin.measure.config.params.env._ +//import org.apache.griffin.measure.config.params.user._ +//import org.apache.griffin.measure.config.reader._ +//import org.apache.griffin.measure.config.validator._ +//import org.apache.griffin.measure.connector.{DataConnector, DataConnectorFactory} +//import org.apache.griffin.measure.log.Loggable +//import org.apache.griffin.measure.rule.expr._ +//import org.apache.griffin.measure.rule.{ExprValueUtil, RuleAnalyzer, RuleFactory} //import org.apache.spark.rdd.RDD //import org.apache.spark.sql.SQLContext //import org.apache.spark.{SparkConf, SparkContext} diff --git a/measure/src/test/scala/org/apache/griffin/measure/batch/algo/BatchProfileAlgoTest.scala b/measure/src/test/scala/org/apache/griffin/measure/algo/BatchProfileAlgoTest.scala similarity index 88% rename from measure/src/test/scala/org/apache/griffin/measure/batch/algo/BatchProfileAlgoTest.scala rename to measure/src/test/scala/org/apache/griffin/measure/algo/BatchProfileAlgoTest.scala index ce59af022..be419a585 100644 --- a/measure/src/test/scala/org/apache/griffin/measure/batch/algo/BatchProfileAlgoTest.scala +++ b/measure/src/test/scala/org/apache/griffin/measure/algo/BatchProfileAlgoTest.scala @@ -12,19 +12,19 @@ //limitations under the License. // // */ -//package org.apache.griffin.measure.batch.algo +//package org.apache.griffin.measure.algo // //import java.util.Date // -//import org.apache.griffin.measure.batch.config.params._ -//import org.apache.griffin.measure.batch.config.params.env._ -//import org.apache.griffin.measure.batch.config.params.user._ -//import org.apache.griffin.measure.batch.config.reader._ -//import org.apache.griffin.measure.batch.config.validator._ -//import org.apache.griffin.measure.batch.connector.{DataConnector, DataConnectorFactory} -//import org.apache.griffin.measure.batch.log.Loggable -//import org.apache.griffin.measure.batch.rule.expr._ -//import org.apache.griffin.measure.batch.rule.{ExprValueUtil, RuleAnalyzer, RuleFactory} +//import org.apache.griffin.measure.config.params._ +//import org.apache.griffin.measure.config.params.env._ +//import org.apache.griffin.measure.config.params.user._ +//import org.apache.griffin.measure.config.reader._ +//import org.apache.griffin.measure.config.validator._ +//import org.apache.griffin.measure.connector.{DataConnector, DataConnectorFactory} +//import org.apache.griffin.measure.log.Loggable +//import org.apache.griffin.measure.rule.expr._ +//import org.apache.griffin.measure.rule.{ExprValueUtil, RuleAnalyzer, RuleFactory} //import org.apache.spark.rdd.RDD //import org.apache.spark.sql.SQLContext //import org.apache.spark.{SparkConf, SparkContext} diff --git a/measure/src/test/scala/org/apache/griffin/measure/batch/algo/core/AccuracyCoreTest.scala b/measure/src/test/scala/org/apache/griffin/measure/algo/core/AccuracyCoreTest.scala similarity index 92% rename from measure/src/test/scala/org/apache/griffin/measure/batch/algo/core/AccuracyCoreTest.scala rename to measure/src/test/scala/org/apache/griffin/measure/algo/core/AccuracyCoreTest.scala index ad109d2b0..5b3c73cf7 100644 --- a/measure/src/test/scala/org/apache/griffin/measure/batch/algo/core/AccuracyCoreTest.scala +++ b/measure/src/test/scala/org/apache/griffin/measure/algo/core/AccuracyCoreTest.scala @@ -12,11 +12,11 @@ See the License for the specific language governing permissions and limitations under the License. */ -package org.apache.griffin.measure.batch.algo.core +package org.apache.griffin.measure.algo.core -import org.apache.griffin.measure.batch.config.params.user.EvaluateRuleParam -import org.apache.griffin.measure.batch.rule.expr._ -import org.apache.griffin.measure.batch.rule.{RuleAnalyzer, RuleFactory} +import org.apache.griffin.measure.config.params.user.EvaluateRuleParam +import org.apache.griffin.measure.rule.expr._ +import org.apache.griffin.measure.rule.{RuleAnalyzer, RuleFactory} import org.junit.runner.RunWith import org.scalatest.junit.JUnitRunner import org.scalatest.{BeforeAndAfter, FunSuite, Matchers} diff --git a/measure/src/test/scala/org/apache/griffin/measure/batch/algo/core/ProfileCoreTest.scala b/measure/src/test/scala/org/apache/griffin/measure/algo/core/ProfileCoreTest.scala similarity index 91% rename from measure/src/test/scala/org/apache/griffin/measure/batch/algo/core/ProfileCoreTest.scala rename to measure/src/test/scala/org/apache/griffin/measure/algo/core/ProfileCoreTest.scala index e1eeb1122..1a697f8bd 100644 --- a/measure/src/test/scala/org/apache/griffin/measure/batch/algo/core/ProfileCoreTest.scala +++ b/measure/src/test/scala/org/apache/griffin/measure/algo/core/ProfileCoreTest.scala @@ -12,11 +12,11 @@ See the License for the specific language governing permissions and limitations under the License. */ -package org.apache.griffin.measure.batch.algo.core +package org.apache.griffin.measure.algo.core -import org.apache.griffin.measure.batch.config.params.user.EvaluateRuleParam -import org.apache.griffin.measure.batch.rule.expr._ -import org.apache.griffin.measure.batch.rule.{RuleAnalyzer, RuleFactory} +import org.apache.griffin.measure.config.params.user.EvaluateRuleParam +import org.apache.griffin.measure.rule.expr._ +import org.apache.griffin.measure.rule.{RuleAnalyzer, RuleFactory} import org.junit.runner.RunWith import org.scalatest.junit.JUnitRunner import org.scalatest.{BeforeAndAfter, FunSuite, Matchers} diff --git a/measure/src/test/scala/org/apache/griffin/measure/batch/config/reader/ParamRawStringReaderTest.scala b/measure/src/test/scala/org/apache/griffin/measure/config/reader/ParamRawStringReaderTest.scala similarity index 90% rename from measure/src/test/scala/org/apache/griffin/measure/batch/config/reader/ParamRawStringReaderTest.scala rename to measure/src/test/scala/org/apache/griffin/measure/config/reader/ParamRawStringReaderTest.scala index 2b46c720e..757c6403d 100644 --- a/measure/src/test/scala/org/apache/griffin/measure/batch/config/reader/ParamRawStringReaderTest.scala +++ b/measure/src/test/scala/org/apache/griffin/measure/config/reader/ParamRawStringReaderTest.scala @@ -12,9 +12,9 @@ See the License for the specific language governing permissions and limitations under the License. */ -package org.apache.griffin.measure.batch.config.reader +package org.apache.griffin.measure.config.reader -import org.apache.griffin.measure.batch.config.params.env._ +import org.apache.griffin.measure.config.params.env._ import org.junit.runner.RunWith import org.scalatest.junit.JUnitRunner import org.scalatest.{BeforeAndAfter, FunSuite, Matchers} diff --git a/measure/src/test/scala/org/apache/griffin/measure/batch/config/validator/AllParamValidatorTest.scala b/measure/src/test/scala/org/apache/griffin/measure/config/validator/AllParamValidatorTest.scala similarity index 90% rename from measure/src/test/scala/org/apache/griffin/measure/batch/config/validator/AllParamValidatorTest.scala rename to measure/src/test/scala/org/apache/griffin/measure/config/validator/AllParamValidatorTest.scala index 923746df8..12963054e 100644 --- a/measure/src/test/scala/org/apache/griffin/measure/batch/config/validator/AllParamValidatorTest.scala +++ b/measure/src/test/scala/org/apache/griffin/measure/config/validator/AllParamValidatorTest.scala @@ -12,9 +12,9 @@ See the License for the specific language governing permissions and limitations under the License. */ -package org.apache.griffin.measure.batch.config.validator +package org.apache.griffin.measure.config.validator -import org.apache.griffin.measure.batch.config.params._ +import org.apache.griffin.measure.config.params._ import org.junit.runner.RunWith import org.scalatest.junit.JUnitRunner import org.scalatest.{BeforeAndAfter, FlatSpec, Matchers} diff --git a/measure/src/test/scala/org/apache/griffin/measure/batch/persist/HdfsPersistTest.scala b/measure/src/test/scala/org/apache/griffin/measure/persist/HdfsPersistTest.scala similarity index 96% rename from measure/src/test/scala/org/apache/griffin/measure/batch/persist/HdfsPersistTest.scala rename to measure/src/test/scala/org/apache/griffin/measure/persist/HdfsPersistTest.scala index 327632ab4..e9d3f92b3 100644 --- a/measure/src/test/scala/org/apache/griffin/measure/batch/persist/HdfsPersistTest.scala +++ b/measure/src/test/scala/org/apache/griffin/measure/persist/HdfsPersistTest.scala @@ -12,7 +12,7 @@ See the License for the specific language governing permissions and limitations under the License. */ -package org.apache.griffin.measure.batch.persist +package org.apache.griffin.measure.persist import org.junit.runner.RunWith import org.scalatest.junit.JUnitRunner diff --git a/measure/src/test/scala/org/apache/griffin/measure/batch/persist/HttpPersistTest.scala b/measure/src/test/scala/org/apache/griffin/measure/persist/HttpPersistTest.scala similarity index 96% rename from measure/src/test/scala/org/apache/griffin/measure/batch/persist/HttpPersistTest.scala rename to measure/src/test/scala/org/apache/griffin/measure/persist/HttpPersistTest.scala index 7f2b4b5db..14b9e3ec0 100644 --- a/measure/src/test/scala/org/apache/griffin/measure/batch/persist/HttpPersistTest.scala +++ b/measure/src/test/scala/org/apache/griffin/measure/persist/HttpPersistTest.scala @@ -12,7 +12,7 @@ See the License for the specific language governing permissions and limitations under the License. */ -package org.apache.griffin.measure.batch.persist +package org.apache.griffin.measure.persist import org.junit.runner.RunWith import org.scalatest.junit.JUnitRunner diff --git a/measure/src/test/scala/org/apache/griffin/measure/batch/result/AccuracyResultTest.scala b/measure/src/test/scala/org/apache/griffin/measure/result/AccuracyResultTest.scala similarity index 97% rename from measure/src/test/scala/org/apache/griffin/measure/batch/result/AccuracyResultTest.scala rename to measure/src/test/scala/org/apache/griffin/measure/result/AccuracyResultTest.scala index b07d872aa..33e59c350 100644 --- a/measure/src/test/scala/org/apache/griffin/measure/batch/result/AccuracyResultTest.scala +++ b/measure/src/test/scala/org/apache/griffin/measure/result/AccuracyResultTest.scala @@ -12,7 +12,7 @@ See the License for the specific language governing permissions and limitations under the License. */ -package org.apache.griffin.measure.batch.result +package org.apache.griffin.measure.result import org.junit.runner.RunWith import org.scalatest.junit.JUnitRunner diff --git a/measure/src/test/scala/org/apache/griffin/measure/batch/result/ProfileResultTest.scala b/measure/src/test/scala/org/apache/griffin/measure/result/ProfileResultTest.scala similarity index 97% rename from measure/src/test/scala/org/apache/griffin/measure/batch/result/ProfileResultTest.scala rename to measure/src/test/scala/org/apache/griffin/measure/result/ProfileResultTest.scala index 7fa3599bb..10fc33d77 100644 --- a/measure/src/test/scala/org/apache/griffin/measure/batch/result/ProfileResultTest.scala +++ b/measure/src/test/scala/org/apache/griffin/measure/result/ProfileResultTest.scala @@ -12,7 +12,7 @@ See the License for the specific language governing permissions and limitations under the License. */ -package org.apache.griffin.measure.batch.result +package org.apache.griffin.measure.result import org.junit.runner.RunWith import org.scalatest.junit.JUnitRunner diff --git a/measure/src/test/scala/org/apache/griffin/measure/batch/rule/RuleAnalyzerTest.scala b/measure/src/test/scala/org/apache/griffin/measure/rule/RuleAnalyzerTest.scala similarity index 95% rename from measure/src/test/scala/org/apache/griffin/measure/batch/rule/RuleAnalyzerTest.scala rename to measure/src/test/scala/org/apache/griffin/measure/rule/RuleAnalyzerTest.scala index 4c4fdeff0..07a435837 100644 --- a/measure/src/test/scala/org/apache/griffin/measure/batch/rule/RuleAnalyzerTest.scala +++ b/measure/src/test/scala/org/apache/griffin/measure/rule/RuleAnalyzerTest.scala @@ -12,9 +12,9 @@ See the License for the specific language governing permissions and limitations under the License. */ -package org.apache.griffin.measure.batch.rule +package org.apache.griffin.measure.rule -import org.apache.griffin.measure.batch.config.params.user.EvaluateRuleParam +import org.apache.griffin.measure.config.params.user.EvaluateRuleParam import org.junit.runner.RunWith import org.scalatest.junit.JUnitRunner import org.scalatest.{BeforeAndAfter, FunSuite, Matchers} diff --git a/measure/src/test/scala/org/apache/griffin/measure/batch/rule/RuleFactoryTest.scala b/measure/src/test/scala/org/apache/griffin/measure/rule/RuleFactoryTest.scala similarity index 92% rename from measure/src/test/scala/org/apache/griffin/measure/batch/rule/RuleFactoryTest.scala rename to measure/src/test/scala/org/apache/griffin/measure/rule/RuleFactoryTest.scala index 17640d7b0..7ca1cb35c 100644 --- a/measure/src/test/scala/org/apache/griffin/measure/batch/rule/RuleFactoryTest.scala +++ b/measure/src/test/scala/org/apache/griffin/measure/rule/RuleFactoryTest.scala @@ -12,9 +12,9 @@ See the License for the specific language governing permissions and limitations under the License. */ -package org.apache.griffin.measure.batch.rule +package org.apache.griffin.measure.rule -import org.apache.griffin.measure.batch.config.params.user.EvaluateRuleParam +import org.apache.griffin.measure.config.params.user.EvaluateRuleParam import org.junit.runner.RunWith import org.scalatest.junit.JUnitRunner import org.scalatest.{BeforeAndAfter, FunSuite, Matchers} diff --git a/measure/src/test/scala/org/apache/griffin/measure/batch/rule/RuleParserTest.scala b/measure/src/test/scala/org/apache/griffin/measure/rule/RuleParserTest.scala similarity index 99% rename from measure/src/test/scala/org/apache/griffin/measure/batch/rule/RuleParserTest.scala rename to measure/src/test/scala/org/apache/griffin/measure/rule/RuleParserTest.scala index b52dde647..cbc95e145 100644 --- a/measure/src/test/scala/org/apache/griffin/measure/batch/rule/RuleParserTest.scala +++ b/measure/src/test/scala/org/apache/griffin/measure/rule/RuleParserTest.scala @@ -12,7 +12,7 @@ See the License for the specific language governing permissions and limitations under the License. */ -package org.apache.griffin.measure.batch.rule +package org.apache.griffin.measure.rule import org.junit.runner.RunWith import org.scalatest.junit.JUnitRunner diff --git a/measure/src/test/scala/org/apache/griffin/measure/batch/utils/JsonUtilTest.scala b/measure/src/test/scala/org/apache/griffin/measure/utils/JsonUtilTest.scala similarity index 97% rename from measure/src/test/scala/org/apache/griffin/measure/batch/utils/JsonUtilTest.scala rename to measure/src/test/scala/org/apache/griffin/measure/utils/JsonUtilTest.scala index a6ba68cf9..9414d87d3 100644 --- a/measure/src/test/scala/org/apache/griffin/measure/batch/utils/JsonUtilTest.scala +++ b/measure/src/test/scala/org/apache/griffin/measure/utils/JsonUtilTest.scala @@ -12,7 +12,7 @@ See the License for the specific language governing permissions and limitations under the License. */ -package org.apache.griffin.measure.batch.utils +package org.apache.griffin.measure.utils import org.junit.runner.RunWith import org.scalatest.junit.JUnitRunner diff --git a/service/src/main/resources/sparkJob.properties b/service/src/main/resources/sparkJob.properties index 0d2cb27e7..51969ac31 100644 --- a/service/src/main/resources/sparkJob.properties +++ b/service/src/main/resources/sparkJob.properties @@ -1,5 +1,5 @@ sparkJob.file=hdfs:///griffin/griffin-measure.jar -sparkJob.className=org.apache.griffin.measure.batch.Application +sparkJob.className=org.apache.griffin.measure.Application sparkJob.args_1=hdfs:///griffin/json/env.json sparkJob.args_3=hdfs,raw sparkJob.name=griffin From 19177dfeb3fb79b21c3493dfc0378ee42f72b66e Mon Sep 17 00:00:00 2001 From: Lionel Liu Date: Tue, 13 Jun 2017 16:59:00 +0800 Subject: [PATCH 004/111] rm -0700 --- ui/index.html | 4 ++-- ui/js/controllers/dataassets-ctrl.js | 2 +- ui/js/controllers/rule-ctrl.js | 2 +- ui/pages/dataassets/dataassets.html | 4 ++-- ui/pages/jobs/jobs.html | 4 ++-- ui/pages/measures/measures.html | 2 +- ui/pages/metrics/download-sample.html | 2 +- ui/sidebar.html | 2 +- 8 files changed, 11 insertions(+), 11 deletions(-) diff --git a/ui/index.html b/ui/index.html index 229401fb3..3b10ae791 100644 --- a/ui/index.html +++ b/ui/index.html @@ -134,9 +134,9 @@
-

{{timestamp|date:'short':'-0700'}} - Welcome !

+

{{timestamp|date:'short'}} - Welcome !

- {{r.timestamp|date:'short':'-0700'}} - {{r.operation}}d a {{r.target}} named + {{r.timestamp|date:'short'}} - {{r.operation}}d a {{r.target}} named {{r.name}}

diff --git a/ui/js/controllers/dataassets-ctrl.js b/ui/js/controllers/dataassets-ctrl.js index 999c7acf8..4bd497e99 100644 --- a/ui/js/controllers/dataassets-ctrl.js +++ b/ui/js/controllers/dataassets-ctrl.js @@ -70,7 +70,7 @@ define(['./module'], function (controllers) { }; var findValue = function(keyword, assetItem) { - var date = $filter('date')(assetItem.timestamp, 'M/d/yy h:mm a', '-0700') + var date = $filter('date')(assetItem.timestamp, 'M/d/yy h:mm a') return include(keyword, assetItem.assetName) || include(keyword, assetItem.assetType) || include(keyword, assetItem.owner) diff --git a/ui/js/controllers/rule-ctrl.js b/ui/js/controllers/rule-ctrl.js index bda015b98..9f3deba16 100644 --- a/ui/js/controllers/rule-ctrl.js +++ b/ui/js/controllers/rule-ctrl.js @@ -64,7 +64,7 @@ define(['./module'], function (controllers) { }; var findValue = function(keyword, assetItem) { - var date = $filter('date')(assetItem.createDate, 'M/d/yy h:mm a', '-0700') + var date = $filter('date')(assetItem.createDate, 'M/d/yy h:mm a') return include(keyword, assetItem.name) || include(keyword, assetItem.description) || include(keyword, assetItem.owner) diff --git a/ui/pages/dataassets/dataassets.html b/ui/pages/dataassets/dataassets.html index 79a3df26a..4ed697bf3 100644 --- a/ui/pages/dataassets/dataassets.html +++ b/ui/pages/dataassets/dataassets.html @@ -43,8 +43,8 @@ {{row.tableName}} {{row.dbName || 'N/A'}} {{row.owner || 'N/A'}} - {{(row.createTime | date: 'short':'-0700') || 'N/A'}} - {{(row.lastAccessTime | date: 'short':'-0700') || 'N/A' }} + {{(row.createTime | date: 'short') || 'N/A'}} + {{(row.lastAccessTime | date: 'short') || 'N/A' }} {{row.sd.location || 'N/A'}} diff --git a/ui/pages/jobs/jobs.html b/ui/pages/jobs/jobs.html index 4f5b6ba0e..b6fc4bbe0 100644 --- a/ui/pages/jobs/jobs.html +++ b/ui/pages/jobs/jobs.html @@ -43,9 +43,9 @@ {{row.jobName}} {{row.sourcePat}} {{row.targetPat|strmap:'modeltype'}} - {{(row.previousFireTime | date: 'short':'-0700') || 'N/A' }} + {{(row.previousFireTime | date: 'short') || 'N/A' }} - {{(row.nextFireTime | date: 'short':'-0700') || 'N/A' }} + {{(row.nextFireTime | date: 'short') || 'N/A' }} {{row.triggerState}} {{row.periodTime}} diff --git a/ui/pages/measures/measures.html b/ui/pages/measures/measures.html index af203234f..9831cb054 100644 --- a/ui/pages/measures/measures.html +++ b/ui/pages/measures/measures.html @@ -47,7 +47,7 @@ {{row.type|strmap:'modeltype'}} {{row.description}} - + {{row.organization}} diff --git a/ui/pages/metrics/download-sample.html b/ui/pages/metrics/download-sample.html index a438182c3..c346ba624 100644 --- a/ui/pages/metrics/download-sample.html +++ b/ui/pages/metrics/download-sample.html @@ -23,7 +23,7 @@
Download sample
No data
- {{item.date | date: 'MM/dd/yy HH:mm':'-0700'}} + {{item.date | date: 'MM/dd/yy HH:mm'}}    diff --git a/ui/sidebar.html b/ui/sidebar.html index 4b4f01146..777b5d13a 100644 --- a/ui/sidebar.html +++ b/ui/sidebar.html @@ -61,7 +61,7 @@

- {{items.timestamp | date:'MM/dd HH:mm':'-0700'}} + {{items.timestamp | date:'MM/dd HH:mm'}}    {{items.name |strShorten}} {{items.dq | number:2}}% From ca5631a64365f703476223b88f55067af5229537 Mon Sep 17 00:00:00 2001 From: Lionel Liu Date: Wed, 14 Jun 2017 09:28:34 +0800 Subject: [PATCH 005/111] license of file --- .../core/schedule/SchedulerService.java | 18 +++++++++++++++--- .../core/metric/MetricServiceImplTest.java | 18 +++++++++++++++--- .../core/schedule/SchedulerControllerTest.java | 18 +++++++++++++++--- 3 files changed, 45 insertions(+), 9 deletions(-) diff --git a/service/src/main/java/org/apache/griffin/core/schedule/SchedulerService.java b/service/src/main/java/org/apache/griffin/core/schedule/SchedulerService.java index 5d57ba19f..d9ff62aef 100644 --- a/service/src/main/java/org/apache/griffin/core/schedule/SchedulerService.java +++ b/service/src/main/java/org/apache/griffin/core/schedule/SchedulerService.java @@ -1,3 +1,18 @@ +/*- + * Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. + + */ + package org.apache.griffin.core.schedule; import org.quartz.SchedulerException; @@ -6,9 +21,6 @@ import java.util.List; import java.util.Map; -/** - * Created by xiangrchen on 6/7/17. - */ public interface SchedulerService { public List> getJobs() throws SchedulerException; diff --git a/service/src/test/java/org/apache/griffin/core/metric/MetricServiceImplTest.java b/service/src/test/java/org/apache/griffin/core/metric/MetricServiceImplTest.java index 671808afd..e8acdf257 100644 --- a/service/src/test/java/org/apache/griffin/core/metric/MetricServiceImplTest.java +++ b/service/src/test/java/org/apache/griffin/core/metric/MetricServiceImplTest.java @@ -1,3 +1,18 @@ +/*- + * Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. + + */ + package org.apache.griffin.core.metric; import org.apache.griffin.core.measure.repo.MeasureRepo; @@ -13,9 +28,6 @@ import static org.assertj.core.api.Assertions.fail; import static org.junit.Assert.assertTrue; -/** - * Created by xiangrchen on 6/7/17. - */ @RunWith(SpringRunner.class) public class MetricServiceImplTest { @TestConfiguration diff --git a/service/src/test/java/org/apache/griffin/core/schedule/SchedulerControllerTest.java b/service/src/test/java/org/apache/griffin/core/schedule/SchedulerControllerTest.java index c4eb3eccb..f1d40d02d 100644 --- a/service/src/test/java/org/apache/griffin/core/schedule/SchedulerControllerTest.java +++ b/service/src/test/java/org/apache/griffin/core/schedule/SchedulerControllerTest.java @@ -1,3 +1,18 @@ +/*- + * Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. + + */ + package org.apache.griffin.core.schedule; import org.codehaus.jackson.map.ObjectMapper; @@ -22,9 +37,6 @@ import static org.springframework.test.web.servlet.result.MockMvcResultMatchers.jsonPath; import static org.springframework.test.web.servlet.result.MockMvcResultMatchers.status; -/** - * Created by xiangrchen on 6/7/17. - */ @RunWith(SpringRunner.class) @WebMvcTest(value = SchedulerController.class,secure = false) public class SchedulerControllerTest { From ea2d28b2d09f1ec29f316d92e59e9de2a217f622 Mon Sep 17 00:00:00 2001 From: Lionel Liu Date: Tue, 27 Jun 2017 14:30:41 +0800 Subject: [PATCH 006/111] v1 --- .../src/main/resources/config-streaming.json | 26 ++-- .../algo/batch/BatchAccuracyAlgo.scala | 8 +- .../measure/algo/batch/BatchProfileAlgo.scala | 6 +- .../streaming/StreamingAccuracyAlgo.scala | 130 ++++++++++++++++++ ...tor.scala => AvroBatchDataConnector.scala} | 6 +- .../connector/BatchDataConnector.scala | 28 ++++ .../measure/connector/DataConnector.scala | 8 -- .../connector/DataConnectorFactory.scala | 69 +++++++++- ...tor.scala => HiveBatchDataConnector.scala} | 6 +- .../connector/KafkaDataConnector.scala | 69 ++++++++++ .../KafkaStreamingDataConnector.scala | 89 ++++++++++++ .../connector/StreamingDataConnector.scala | 29 ++++ .../measure/connector/ConnectorTest.scala | 40 ++++++ 13 files changed, 476 insertions(+), 38 deletions(-) create mode 100644 measure/src/main/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgo.scala rename measure/src/main/scala/org/apache/griffin/measure/connector/{AvroDataConnector.scala => AvroBatchDataConnector.scala} (92%) create mode 100644 measure/src/main/scala/org/apache/griffin/measure/connector/BatchDataConnector.scala rename measure/src/main/scala/org/apache/griffin/measure/connector/{HiveDataConnector.scala => HiveBatchDataConnector.scala} (94%) create mode 100644 measure/src/main/scala/org/apache/griffin/measure/connector/KafkaDataConnector.scala create mode 100644 measure/src/main/scala/org/apache/griffin/measure/connector/KafkaStreamingDataConnector.scala create mode 100644 measure/src/main/scala/org/apache/griffin/measure/connector/StreamingDataConnector.scala create mode 100644 measure/src/test/scala/org/apache/griffin/measure/connector/ConnectorTest.scala diff --git a/measure/src/main/resources/config-streaming.json b/measure/src/main/resources/config-streaming.json index fe8877736..3ba458c65 100644 --- a/measure/src/main/resources/config-streaming.json +++ b/measure/src/main/resources/config-streaming.json @@ -6,11 +6,14 @@ "type": "kafka", "version": "0.8", "config": { - "bootstrap.servers": "a.b.c.d:9092", - "group.id": "group1", - "auto.offset.reset": "smallest", - "auto.commit.enable": "false", - "topics": "src" + "kafka.config": { + "bootstrap.servers": "a.b.c.d:9092", + "group.id": "group1", + "auto.offset.reset": "smallest", + "auto.commit.enable": "false", + }, + "topics": "src", + "decoder": "kafka.serializer.StringDecoder" } }, @@ -18,11 +21,14 @@ "type": "kafka", "version": "0.8", "config": { - "bootstrap.servers": "a.b.c.d:9092", - "group.id": "group1", - "auto.offset.reset": "smallest", - "auto.commit.enable": "false", - "topics": "tgt" + "kafka.config": { + "bootstrap.servers": "a.b.c.d:9092", + "group.id": "group1", + "auto.offset.reset": "smallest", + "auto.commit.enable": "false", + }, + "topics": "tgt", + "decoder": "kafka.serializer.StringDecoder" } }, diff --git a/measure/src/main/scala/org/apache/griffin/measure/algo/batch/BatchAccuracyAlgo.scala b/measure/src/main/scala/org/apache/griffin/measure/algo/batch/BatchAccuracyAlgo.scala index 8debc0d82..a29a5f14c 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/algo/batch/BatchAccuracyAlgo.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/algo/batch/BatchAccuracyAlgo.scala @@ -65,8 +65,8 @@ case class BatchAccuracyAlgo(allParam: AllParam) extends AccuracyAlgo { val finalConstExprValueMap = ExprValueUtil.updateExprValueMap(ruleAnalyzer.constFinalCacheExprs, constExprValueMap) // data connector - val sourceDataConnector: DataConnector = - DataConnectorFactory.getDataConnector(sqlContext, userParam.sourceParam, + val sourceDataConnector: BatchDataConnector = + DataConnectorFactory.getBatchDataConnector(sqlContext, userParam.sourceParam, ruleAnalyzer.sourceRuleExprs, finalConstExprValueMap ) match { case Success(cntr) => { @@ -75,8 +75,8 @@ case class BatchAccuracyAlgo(allParam: AllParam) extends AccuracyAlgo { } case Failure(ex) => throw ex } - val targetDataConnector: DataConnector = - DataConnectorFactory.getDataConnector(sqlContext, userParam.targetParam, + val targetDataConnector: BatchDataConnector = + DataConnectorFactory.getBatchDataConnector(sqlContext, userParam.targetParam, ruleAnalyzer.targetRuleExprs, finalConstExprValueMap ) match { case Success(cntr) => { diff --git a/measure/src/main/scala/org/apache/griffin/measure/algo/batch/BatchProfileAlgo.scala b/measure/src/main/scala/org/apache/griffin/measure/algo/batch/BatchProfileAlgo.scala index 645459ec4..c7b33c583 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/algo/batch/BatchProfileAlgo.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/algo/batch/BatchProfileAlgo.scala @@ -19,7 +19,7 @@ import java.util.Date import org.apache.griffin.measure.algo.ProfileAlgo import org.apache.griffin.measure.algo.core.ProfileCore import org.apache.griffin.measure.config.params._ -import org.apache.griffin.measure.connector.{DataConnector, DataConnectorFactory} +import org.apache.griffin.measure.connector._ import org.apache.griffin.measure.persist.{Persist, PersistFactory} import org.apache.griffin.measure.result._ import org.apache.griffin.measure.rule.expr.{Expr, StatementExpr} @@ -65,8 +65,8 @@ case class BatchProfileAlgo(allParam: AllParam) extends ProfileAlgo { val finalConstExprValueMap = ExprValueUtil.updateExprValueMap(ruleAnalyzer.constFinalCacheExprs, constExprValueMap) // data connector - val sourceDataConnector: DataConnector = - DataConnectorFactory.getDataConnector(sqlContext, userParam.sourceParam, + val sourceDataConnector: BatchDataConnector = + DataConnectorFactory.getBatchDataConnector(sqlContext, userParam.sourceParam, ruleAnalyzer.sourceRuleExprs, finalConstExprValueMap ) match { case Success(cntr) => { diff --git a/measure/src/main/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgo.scala b/measure/src/main/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgo.scala new file mode 100644 index 000000000..2db7cd49a --- /dev/null +++ b/measure/src/main/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgo.scala @@ -0,0 +1,130 @@ +///*- +// * Licensed under the Apache License, Version 2.0 (the "License"); +//you may not use this file except in compliance with the License. +//You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +//Unless required by applicable law or agreed to in writing, software +//distributed under the License is distributed on an "AS IS" BASIS, +//WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +//See the License for the specific language governing permissions and +//limitations under the License. +// +// */ +//package org.apache.griffin.measure.algo.streaming +// +//import java.util.Date +// +//import org.apache.griffin.measure.algo.AccuracyAlgo +//import org.apache.griffin.measure.config.params.AllParam +//import org.apache.griffin.measure.connector.{DataConnector, DataConnectorFactory} +//import org.apache.griffin.measure.persist.{Persist, PersistFactory} +//import org.apache.griffin.measure.rule.{ExprValueUtil, RuleAnalyzer, RuleFactory} +//import org.apache.griffin.measure.rule.expr.StatementExpr +//import org.apache.spark.sql.hive.HiveContext +//import org.apache.spark.{SparkConf, SparkContext} +// +//import scala.util.{Try, Success, Failure} +// +// +//case class StreamingAccuracyAlgo(allParam: AllParam) extends AccuracyAlgo { +// val envParam = allParam.envParam +// val userParam = allParam.userParam +// +// def run(): Try[_] = { +// Try { +// val metricName = userParam.name +// +// val conf = new SparkConf().setAppName(metricName) +// val sc = new SparkContext(conf) +// val sqlContext = new HiveContext(sc) +// +// // start time +// val startTime = new Date().getTime() +// +// // get persists to persist measure result +// val persist: Persist = PersistFactory(envParam.persistParams, metricName).getPersists(startTime) +// +// // get spark application id +// val applicationId = sc.applicationId +// +// // persist start id +// persist.start(applicationId) +// +// // generate rule from rule param, generate rule analyzer +// val ruleFactory = RuleFactory(userParam.evaluateRuleParam) +// val rule: StatementExpr = ruleFactory.generateRule() +// val ruleAnalyzer: RuleAnalyzer = RuleAnalyzer(rule) +// +// // const expr value map +// val constExprValueMap = ExprValueUtil.genExprValueMap(None, ruleAnalyzer.constCacheExprs, Map[String, Any]()) +// val finalConstExprValueMap = ExprValueUtil.updateExprValueMap(ruleAnalyzer.constFinalCacheExprs, constExprValueMap) +// +// // data connector +// val sourceDataConnector: DataConnector = +// DataConnectorFactory.getStreamingDataConnector(sqlContext, userParam.sourceParam, +// ruleAnalyzer.sourceRuleExprs, finalConstExprValueMap +// ) match { +// case Success(cntr) => { +// if (cntr.available) cntr +// else throw new Exception("source data connection error!") +// } +// case Failure(ex) => throw ex +// } +// val targetDataConnector: DataConnector = +// DataConnectorFactory.getStreamingDataConnector(sqlContext, userParam.targetParam, +// ruleAnalyzer.targetRuleExprs, finalConstExprValueMap +// ) match { +// case Success(cntr) => { +// if (cntr.available) cntr +// else throw new Exception("target data connection error!") +// } +// case Failure(ex) => throw ex +// } +// +// // get metadata +// // val sourceMetaData: Iterable[(String, String)] = sourceDataConnector.metaData() match { +// // case Success(md) => md +// // case _ => throw new Exception("source metadata error!") +// // } +// // val targetMetaData: Iterable[(String, String)] = targetDataConnector.metaData() match { +// // case Success(md) => md +// // case _ => throw new Exception("target metadata error!") +// // } +// +// // get data +// val sourceData: RDD[(Product, Map[String, Any])] = sourceDataConnector.data() match { +// case Success(dt) => dt +// case Failure(ex) => throw ex +// } +// val targetData: RDD[(Product, Map[String, Any])] = targetDataConnector.data() match { +// case Success(dt) => dt +// case Failure(ex) => throw ex +// } +// +// // accuracy algorithm +// val (accuResult, missingRdd, matchedRdd) = accuracy(sourceData, targetData, ruleAnalyzer) +// +// // end time +// val endTime = new Date().getTime +// persist.log(endTime, s"calculation using time: ${endTime - startTime} ms") +// +// // persist result +// persist.result(endTime, accuResult) +// val missingRecords = missingRdd.map(record2String(_, ruleAnalyzer.sourceRuleExprs.persistExprs, ruleAnalyzer.targetRuleExprs.persistExprs)) +// persist.missRecords(missingRecords) +// +// // persist end time +// val persistEndTime = new Date().getTime +// persist.log(persistEndTime, s"persist using time: ${persistEndTime - endTime} ms") +// +// // finish +// persist.finish() +// +// // context stop +// sc.stop +// } +// } +// +//} diff --git a/measure/src/main/scala/org/apache/griffin/measure/connector/AvroDataConnector.scala b/measure/src/main/scala/org/apache/griffin/measure/connector/AvroBatchDataConnector.scala similarity index 92% rename from measure/src/main/scala/org/apache/griffin/measure/connector/AvroDataConnector.scala rename to measure/src/main/scala/org/apache/griffin/measure/connector/AvroBatchDataConnector.scala index 04ed07d34..4f4f94112 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/connector/AvroDataConnector.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/connector/AvroBatchDataConnector.scala @@ -26,9 +26,9 @@ import org.apache.griffin.measure.rule.{ExprValueUtil, RuleExprs} import org.apache.griffin.measure.utils.HdfsUtil // data connector for avro file -case class AvroDataConnector(sqlContext: SQLContext, config: Map[String, Any], - ruleExprs: RuleExprs, constFinalExprValueMap: Map[String, Any] - ) extends DataConnector { +case class AvroBatchDataConnector(sqlContext: SQLContext, config: Map[String, Any], + ruleExprs: RuleExprs, constFinalExprValueMap: Map[String, Any] + ) extends BatchDataConnector { val FilePath = "file.path" val FileName = "file.name" diff --git a/measure/src/main/scala/org/apache/griffin/measure/connector/BatchDataConnector.scala b/measure/src/main/scala/org/apache/griffin/measure/connector/BatchDataConnector.scala new file mode 100644 index 000000000..12a093c8a --- /dev/null +++ b/measure/src/main/scala/org/apache/griffin/measure/connector/BatchDataConnector.scala @@ -0,0 +1,28 @@ +/*- + * Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. + + */ +package org.apache.griffin.measure.connector + +import org.apache.spark.rdd.RDD + +import scala.util.Try + + +trait BatchDataConnector extends DataConnector { + + def metaData(): Try[Iterable[(String, String)]] + + def data(): Try[RDD[(Product, Map[String, Any])]] + +} diff --git a/measure/src/main/scala/org/apache/griffin/measure/connector/DataConnector.scala b/measure/src/main/scala/org/apache/griffin/measure/connector/DataConnector.scala index 0189cdaed..5b6fefa06 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/connector/DataConnector.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/connector/DataConnector.scala @@ -14,17 +14,9 @@ limitations under the License. */ package org.apache.griffin.measure.connector -import org.apache.spark.rdd.RDD - -import scala.util.Try - trait DataConnector extends Serializable { def available(): Boolean - def metaData(): Try[Iterable[(String, String)]] - - def data(): Try[RDD[(Product, Map[String, Any])]] - } diff --git a/measure/src/main/scala/org/apache/griffin/measure/connector/DataConnectorFactory.scala b/measure/src/main/scala/org/apache/griffin/measure/connector/DataConnectorFactory.scala index 583e86d6a..058ab8991 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/connector/DataConnectorFactory.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/connector/DataConnectorFactory.scala @@ -14,11 +14,16 @@ limitations under the License. */ package org.apache.griffin.measure.connector +import kafka.serializer.StringDecoder import org.apache.griffin.measure.config.params.user._ import org.apache.griffin.measure.rule.RuleExprs import org.apache.griffin.measure.rule.expr._ import org.apache.spark.sql.SQLContext +import org.apache.spark.streaming.StreamingContext +import org.apache.spark.streaming.dstream.InputDStream +import org.apache.spark.streaming.kafka.KafkaUtils +import scala.reflect.ClassTag import scala.util.Try object DataConnectorFactory { @@ -26,20 +31,70 @@ object DataConnectorFactory { val HiveRegex = """^(?i)hive$""".r val AvroRegex = """^(?i)avro$""".r - def getDataConnector(sqlContext: SQLContext, - dataConnectorParam: DataConnectorParam, - ruleExprs: RuleExprs, - globalFinalCacheMap: Map[String, Any] - ): Try[DataConnector] = { + def getBatchDataConnector(sqlContext: SQLContext, + dataConnectorParam: DataConnectorParam, + ruleExprs: RuleExprs, + globalFinalCacheMap: Map[String, Any] + ): Try[BatchDataConnector] = { val conType = dataConnectorParam.conType val version = dataConnectorParam.version + val config = dataConnectorParam.config Try { conType match { - case HiveRegex() => HiveDataConnector(sqlContext, dataConnectorParam.config, ruleExprs, globalFinalCacheMap) - case AvroRegex() => AvroDataConnector(sqlContext, dataConnectorParam.config, ruleExprs, globalFinalCacheMap) + case HiveRegex() => HiveBatchDataConnector(sqlContext, config, ruleExprs, globalFinalCacheMap) + case AvroRegex() => AvroBatchDataConnector(sqlContext, config, ruleExprs, globalFinalCacheMap) case _ => throw new Exception("connector creation error!") } } } + def getStreamingDataConnector(ssc: StreamingContext, + dataConnectorParam: DataConnectorParam, + ruleExprs: RuleExprs, + globalFinalCacheMap: Map[String, Any] + ): Try[StreamingDataConnector] = { + val conType = dataConnectorParam.conType + val version = dataConnectorParam.version + val config = dataConnectorParam.config + Try { + conType match { + case HiveRegex() => { + val KeyType = "key.type" + val ValueType = "value.type" + val keyType = dataConnectorParam.config.getOrElse(KeyType, "java.lang.String").toString + val valueType = dataConnectorParam.config.getOrElse(ValueType, "java.lang.String").toString + (getClassTag(keyType), getClassTag(valueType)) match { + case (ClassTag(k: Class[String]), ClassTag(v: Class[String])) => { + new KafkaDataConnector(ssc, config) { + type K = String + type KD = StringDecoder + type V = String + type VD = StringDecoder + + def stream(): Try[InputDStream[(K, V)]] = Try { + val topicSet = topics.split(",").toSet + KafkaUtils.createDirectStream[K, V, KD, VD]( + ssc, + kafkaConfig, + topicSet + ) + } + } + } + } + } + case _ => throw new Exception("connector creation error!") + } + } + } + + private def getClassTag(tp: String): ClassTag[_] = { + try { + val clazz = Class.forName(tp) + ClassTag(clazz) + } catch { + case e: Throwable => throw e + } + } + } diff --git a/measure/src/main/scala/org/apache/griffin/measure/connector/HiveDataConnector.scala b/measure/src/main/scala/org/apache/griffin/measure/connector/HiveBatchDataConnector.scala similarity index 94% rename from measure/src/main/scala/org/apache/griffin/measure/connector/HiveDataConnector.scala rename to measure/src/main/scala/org/apache/griffin/measure/connector/HiveBatchDataConnector.scala index 41bd073d0..26c6ff1fa 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/connector/HiveDataConnector.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/connector/HiveBatchDataConnector.scala @@ -22,9 +22,9 @@ import org.apache.spark.sql.{DataFrame, Row, SQLContext} import scala.util.{Success, Try} // data connector for hive -case class HiveDataConnector(sqlContext: SQLContext, config: Map[String, Any], - ruleExprs: RuleExprs, constFinalExprValueMap: Map[String, Any] - ) extends DataConnector { +case class HiveBatchDataConnector(sqlContext: SQLContext, config: Map[String, Any], + ruleExprs: RuleExprs, constFinalExprValueMap: Map[String, Any] + ) extends BatchDataConnector { val Database = "database" val TableName = "table.name" diff --git a/measure/src/main/scala/org/apache/griffin/measure/connector/KafkaDataConnector.scala b/measure/src/main/scala/org/apache/griffin/measure/connector/KafkaDataConnector.scala new file mode 100644 index 000000000..5d9d0e40c --- /dev/null +++ b/measure/src/main/scala/org/apache/griffin/measure/connector/KafkaDataConnector.scala @@ -0,0 +1,69 @@ +package org.apache.griffin.measure.connector + +import kafka.serializer.{Decoder, DefaultDecoder, StringDecoder} +import org.apache.griffin.measure.rule.RuleExprs +import org.apache.spark.streaming.StreamingContext +import org.apache.spark.streaming.dstream.InputDStream +import org.apache.spark.streaming.kafka.KafkaUtils + +import scala.reflect.ClassTag +import scala.util.Try + +abstract class KafkaDataConnector(ssc: StreamingContext, config: Map[String, Any] + ) extends StreamingDataConnector { + type KD <: Decoder[K] + type VD <: Decoder[V] + + val KafkaConfig = "kafka.config" + val Topics = "topics" + + val kafkaConfig = config.get(KafkaConfig) match { + case map: Map[String, Any] => map.mapValues(_.toString) + case _ => Map[String, String]() + } + val topics = config.getOrElse(Topics, "").toString + + def available(): Boolean = { + true + } +} + +object KafkaTool { + def generateKafkaDataConnector(ssc: StreamingContext, config: Map[String, Any]): KafkaDataConnector = { + val KeyType = "key.type" + val ValueType = "value.type" + + val keyType = config.getOrElse(KeyType, "java.lang.String").toString + val valueType = config.getOrElse(ValueType, "java.lang.String").toString + + (getClassTag(keyType), getClassTag(valueType)) match { + case (ClassTag(k: Class[String]), ClassTag(v: Class[String])) => { + new KafkaDataConnector(ssc, config) { + type K = String + type KD = StringDecoder + type V = String + type VD = StringDecoder + + def stream(): Try[InputDStream[(K, V)]] = Try { + val topicSet = topics.split(",").toSet + KafkaUtils.createDirectStream[K, V, KD, VD]( + ssc, + kafkaConfig, + topicSet + ) + } + } + } + } + + } + + private def getClassTag(tp: String): ClassTag[_] = { + try { + val clazz = Class.forName(tp) + ClassTag(clazz) + } catch { + case e: Throwable => throw e + } + } +} \ No newline at end of file diff --git a/measure/src/main/scala/org/apache/griffin/measure/connector/KafkaStreamingDataConnector.scala b/measure/src/main/scala/org/apache/griffin/measure/connector/KafkaStreamingDataConnector.scala new file mode 100644 index 000000000..6b865bf0d --- /dev/null +++ b/measure/src/main/scala/org/apache/griffin/measure/connector/KafkaStreamingDataConnector.scala @@ -0,0 +1,89 @@ +///*- +// * Licensed under the Apache License, Version 2.0 (the "License"); +//you may not use this file except in compliance with the License. +//You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +//Unless required by applicable law or agreed to in writing, software +//distributed under the License is distributed on an "AS IS" BASIS, +//WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +//See the License for the specific language governing permissions and +//limitations under the License. +// +// */ +//package org.apache.griffin.measure.connector +// +//import kafka.serializer._ +//import org.apache.griffin.measure.rule.RuleExprs +//import org.apache.spark.streaming.StreamingContext +//import org.apache.spark.streaming.dstream.InputDStream +//import org.apache.spark.streaming.kafka.KafkaUtils +// +//import scala.reflect.ClassTag +// +//import scala.util.Try +// +// +//case class KafkaStreamingDataConnector(ssc: StreamingContext, config: Map[String, Any], +// ruleExprs: RuleExprs, constFinalExprValueMap: Map[String, Any] +// ) extends StreamingDataConnector { +// +// val KafkaConfig = "kafka.config" +// val Topics = "topics" +// val KeyType = "key.type" +// val KeyDeserializer = "key.deserializer" +// val ValueType = "value.type" +// val ValueDeserializer = "value.deserializer" +// +// val kafkaConfig = config.get(KafkaConfig) match { +// case map: Map[String, Any] => map.mapValues(_.toString) +// case _ => Map[String, String]() +// } +// val topics = config.getOrElse(Topics, "").toString +// +// val keyType = config.getOrElse(KeyType, "java.lang.String").toString +// val valueType = config.getOrElse(ValueType, "java.lang.String").toString +// +//// import scala.reflect.runtime.universe._ +//// val m = runtimeMirror(getClass.getClassLoader) +//// val classSymbol = m.staticClass("java.lang.String") +//// val tpe = classSymbol.selfType +// +// def available(): Boolean = { +// true +// } +// +// def stream(): Try[InputDStream[(_, _)]] = Try { +// val topicSet = topics.split(",").toSet +// val keyClassTag = getClassTag(keyType) +// val valueClassTag = getClassTag(valueType) +// KafkaUtils.createDirectStream( +// ssc, +// kafkaConfig, +// topicSet +// )(keyClassTag, valueClassTag, getDeserializer(keyClassTag), getDeserializer(valueClassTag)) +// } +// +// private def getClassTag(tp: String): ClassTag[_] = { +// try { +// val clazz = Class.forName(tp) +// ClassTag(clazz) +// } catch { +// case e: Throwable => throw e +// } +// +// } +// +// private def getDeserializer[T](ct: ClassTag[T]): ClassTag[Decoder[T]] = { +// try { +// ct match { +// case ClassTag(clz: Class[String]) => ClassTag(classOf[StringDecoder]) +// case _ => ClassTag(classOf[DefaultDecoder]) +// } +// } catch { +// case e: Throwable => throw e +// } +// } +// +//} diff --git a/measure/src/main/scala/org/apache/griffin/measure/connector/StreamingDataConnector.scala b/measure/src/main/scala/org/apache/griffin/measure/connector/StreamingDataConnector.scala new file mode 100644 index 000000000..b0bc80e52 --- /dev/null +++ b/measure/src/main/scala/org/apache/griffin/measure/connector/StreamingDataConnector.scala @@ -0,0 +1,29 @@ +/*- + * Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. + + */ +package org.apache.griffin.measure.connector + +import org.apache.spark.streaming.dstream.InputDStream + +import scala.util.Try + + +trait StreamingDataConnector extends DataConnector { + + type K + type V + + def stream(): Try[InputDStream[(K, V)]] + +} diff --git a/measure/src/test/scala/org/apache/griffin/measure/connector/ConnectorTest.scala b/measure/src/test/scala/org/apache/griffin/measure/connector/ConnectorTest.scala new file mode 100644 index 000000000..de56d2c2e --- /dev/null +++ b/measure/src/test/scala/org/apache/griffin/measure/connector/ConnectorTest.scala @@ -0,0 +1,40 @@ +package org.apache.griffin.measure.connector + +import org.apache.griffin.measure.config.params.env._ +import org.apache.griffin.measure.config.reader.ParamRawStringReader +import org.junit.runner.RunWith +import org.scalatest.junit.JUnitRunner +import org.scalatest.{BeforeAndAfter, FunSuite, Matchers} + +import scala.reflect.ClassTag + +@RunWith(classOf[JUnitRunner]) +class ConnectorTest extends FunSuite with Matchers with BeforeAndAfter { + + test("read config") { + + val a = "java.lang.String" + val at = getClassTag(a) + println(at) + + at match { + case ClassTag(m) => println(m) + case _ => println("no") + } + + } + + private def getClassTag(tp: String): ClassTag[_] = { + val clazz = Class.forName(tp) + ClassTag(clazz) + } + +// private def getDeserializer(ct: ClassTag[_]): String = { +// ct.runtimeClass.get +// ct match { +// case Some(t: scala.Predef.Class[String]) => "kafka.serializer.StringDecoder" +// } +// } + +} + From 55ae1dc24d75105be905c0f82b4aa64855a184a4 Mon Sep 17 00:00:00 2001 From: Lionel Liu Date: Tue, 27 Jun 2017 14:59:42 +0800 Subject: [PATCH 007/111] v2 --- .../src/main/resources/config-streaming.json | 6 +- measure/src/main/resources/env.json | 1 + .../streaming/StreamingAccuracyAlgo.scala | 269 +++++++++--------- .../config/params/env/SparkParam.scala | 1 + .../connector/DataConnectorFactory.scala | 56 ++-- .../connector/KafkaDataConnector.scala | 48 +--- .../KafkaStreamingDataConnector.scala | 89 ------ .../measure/rule/expr/LiteralExpr.scala | 24 +- .../griffin/measure/utils/TimeUtil.scala | 47 +++ 9 files changed, 228 insertions(+), 313 deletions(-) delete mode 100644 measure/src/main/scala/org/apache/griffin/measure/connector/KafkaStreamingDataConnector.scala create mode 100644 measure/src/main/scala/org/apache/griffin/measure/utils/TimeUtil.scala diff --git a/measure/src/main/resources/config-streaming.json b/measure/src/main/resources/config-streaming.json index 3ba458c65..e5a83dde3 100644 --- a/measure/src/main/resources/config-streaming.json +++ b/measure/src/main/resources/config-streaming.json @@ -13,7 +13,8 @@ "auto.commit.enable": "false", }, "topics": "src", - "decoder": "kafka.serializer.StringDecoder" + "key.type": "java.lang.String", + "value.type": "java.lang.String" } }, @@ -28,7 +29,8 @@ "auto.commit.enable": "false", }, "topics": "tgt", - "decoder": "kafka.serializer.StringDecoder" + "key.type": "java.lang.String", + "value.type": "java.lang.String" } }, diff --git a/measure/src/main/resources/env.json b/measure/src/main/resources/env.json index 57da8956f..31acc7b6e 100644 --- a/measure/src/main/resources/env.json +++ b/measure/src/main/resources/env.json @@ -2,6 +2,7 @@ "spark": { "log.level": "INFO", "checkpoint.dir": "hdfs:///griffin/batch/cp", + "batch.interval": "10s", "config": {} }, diff --git a/measure/src/main/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgo.scala b/measure/src/main/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgo.scala index 2db7cd49a..9d5bd4de1 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgo.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgo.scala @@ -1,130 +1,139 @@ -///*- -// * Licensed under the Apache License, Version 2.0 (the "License"); -//you may not use this file except in compliance with the License. -//You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -//Unless required by applicable law or agreed to in writing, software -//distributed under the License is distributed on an "AS IS" BASIS, -//WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -//See the License for the specific language governing permissions and -//limitations under the License. -// -// */ -//package org.apache.griffin.measure.algo.streaming -// -//import java.util.Date -// -//import org.apache.griffin.measure.algo.AccuracyAlgo -//import org.apache.griffin.measure.config.params.AllParam -//import org.apache.griffin.measure.connector.{DataConnector, DataConnectorFactory} -//import org.apache.griffin.measure.persist.{Persist, PersistFactory} -//import org.apache.griffin.measure.rule.{ExprValueUtil, RuleAnalyzer, RuleFactory} -//import org.apache.griffin.measure.rule.expr.StatementExpr -//import org.apache.spark.sql.hive.HiveContext -//import org.apache.spark.{SparkConf, SparkContext} -// -//import scala.util.{Try, Success, Failure} -// -// -//case class StreamingAccuracyAlgo(allParam: AllParam) extends AccuracyAlgo { -// val envParam = allParam.envParam -// val userParam = allParam.userParam -// -// def run(): Try[_] = { -// Try { -// val metricName = userParam.name -// -// val conf = new SparkConf().setAppName(metricName) -// val sc = new SparkContext(conf) -// val sqlContext = new HiveContext(sc) -// -// // start time -// val startTime = new Date().getTime() -// -// // get persists to persist measure result -// val persist: Persist = PersistFactory(envParam.persistParams, metricName).getPersists(startTime) -// -// // get spark application id -// val applicationId = sc.applicationId -// -// // persist start id -// persist.start(applicationId) -// -// // generate rule from rule param, generate rule analyzer -// val ruleFactory = RuleFactory(userParam.evaluateRuleParam) -// val rule: StatementExpr = ruleFactory.generateRule() -// val ruleAnalyzer: RuleAnalyzer = RuleAnalyzer(rule) -// -// // const expr value map -// val constExprValueMap = ExprValueUtil.genExprValueMap(None, ruleAnalyzer.constCacheExprs, Map[String, Any]()) -// val finalConstExprValueMap = ExprValueUtil.updateExprValueMap(ruleAnalyzer.constFinalCacheExprs, constExprValueMap) -// -// // data connector -// val sourceDataConnector: DataConnector = -// DataConnectorFactory.getStreamingDataConnector(sqlContext, userParam.sourceParam, -// ruleAnalyzer.sourceRuleExprs, finalConstExprValueMap -// ) match { -// case Success(cntr) => { -// if (cntr.available) cntr -// else throw new Exception("source data connection error!") -// } -// case Failure(ex) => throw ex -// } -// val targetDataConnector: DataConnector = -// DataConnectorFactory.getStreamingDataConnector(sqlContext, userParam.targetParam, -// ruleAnalyzer.targetRuleExprs, finalConstExprValueMap -// ) match { -// case Success(cntr) => { -// if (cntr.available) cntr -// else throw new Exception("target data connection error!") -// } -// case Failure(ex) => throw ex -// } -// -// // get metadata -// // val sourceMetaData: Iterable[(String, String)] = sourceDataConnector.metaData() match { -// // case Success(md) => md -// // case _ => throw new Exception("source metadata error!") -// // } -// // val targetMetaData: Iterable[(String, String)] = targetDataConnector.metaData() match { -// // case Success(md) => md -// // case _ => throw new Exception("target metadata error!") -// // } -// -// // get data -// val sourceData: RDD[(Product, Map[String, Any])] = sourceDataConnector.data() match { -// case Success(dt) => dt -// case Failure(ex) => throw ex -// } -// val targetData: RDD[(Product, Map[String, Any])] = targetDataConnector.data() match { -// case Success(dt) => dt -// case Failure(ex) => throw ex -// } -// -// // accuracy algorithm -// val (accuResult, missingRdd, matchedRdd) = accuracy(sourceData, targetData, ruleAnalyzer) -// -// // end time -// val endTime = new Date().getTime -// persist.log(endTime, s"calculation using time: ${endTime - startTime} ms") -// -// // persist result -// persist.result(endTime, accuResult) -// val missingRecords = missingRdd.map(record2String(_, ruleAnalyzer.sourceRuleExprs.persistExprs, ruleAnalyzer.targetRuleExprs.persistExprs)) -// persist.missRecords(missingRecords) -// -// // persist end time -// val persistEndTime = new Date().getTime -// persist.log(persistEndTime, s"persist using time: ${persistEndTime - endTime} ms") -// -// // finish -// persist.finish() -// -// // context stop -// sc.stop -// } -// } -// -//} +/*- + * Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. + + */ +package org.apache.griffin.measure.algo.streaming + +import java.util.Date + +import org.apache.griffin.measure.algo.AccuracyAlgo +import org.apache.griffin.measure.config.params.AllParam +import org.apache.griffin.measure.connector.{DataConnector, DataConnectorFactory} +import org.apache.griffin.measure.persist.{Persist, PersistFactory} +import org.apache.griffin.measure.rule.{ExprValueUtil, RuleAnalyzer, RuleFactory} +import org.apache.griffin.measure.rule.expr.StatementExpr +import org.apache.griffin.measure.utils.TimeUtil +import org.apache.spark.sql.hive.HiveContext +import org.apache.spark.streaming.{Milliseconds, Seconds, StreamingContext} +import org.apache.spark.{SparkConf, SparkContext} + +import scala.util.{Failure, Success, Try} + + +case class StreamingAccuracyAlgo(allParam: AllParam) extends AccuracyAlgo { + val envParam = allParam.envParam + val userParam = allParam.userParam + + def run(): Try[_] = { + Try { + val metricName = userParam.name + + val conf = new SparkConf().setAppName(metricName) + val sc = new SparkContext(conf) + val sqlContext = new HiveContext(sc) + + val interval = TimeUtil.milliseconds(envParam.sparkParam.batchInterval) match { + case Some(interval) => Milliseconds(interval) + case _ => throw new Exception("invalid batch interval") + } + val ssc = new StreamingContext(sc, interval) + ssc.checkpoint(envParam.sparkParam.cpDir) + + // start time + val startTime = new Date().getTime() + + // get persists to persist measure result + val persist: Persist = PersistFactory(envParam.persistParams, metricName).getPersists(startTime) + + // get spark application id + val applicationId = sc.applicationId + + // persist start id + persist.start(applicationId) + + // generate rule from rule param, generate rule analyzer + val ruleFactory = RuleFactory(userParam.evaluateRuleParam) + val rule: StatementExpr = ruleFactory.generateRule() + val ruleAnalyzer: RuleAnalyzer = RuleAnalyzer(rule) + + // const expr value map + val constExprValueMap = ExprValueUtil.genExprValueMap(None, ruleAnalyzer.constCacheExprs, Map[String, Any]()) + val finalConstExprValueMap = ExprValueUtil.updateExprValueMap(ruleAnalyzer.constFinalCacheExprs, constExprValueMap) + + // data connector + val sourceDataConnector: DataConnector = + DataConnectorFactory.getStreamingDataConnector(ssc, userParam.sourceParam, + ruleAnalyzer.sourceRuleExprs, finalConstExprValueMap + ) match { + case Success(cntr) => { + if (cntr.available) cntr + else throw new Exception("source data connection error!") + } + case Failure(ex) => throw ex + } + val targetDataConnector: DataConnector = + DataConnectorFactory.getStreamingDataConnector(sqlContext, userParam.targetParam, + ruleAnalyzer.targetRuleExprs, finalConstExprValueMap + ) match { + case Success(cntr) => { + if (cntr.available) cntr + else throw new Exception("target data connection error!") + } + case Failure(ex) => throw ex + } + + // get metadata + // val sourceMetaData: Iterable[(String, String)] = sourceDataConnector.metaData() match { + // case Success(md) => md + // case _ => throw new Exception("source metadata error!") + // } + // val targetMetaData: Iterable[(String, String)] = targetDataConnector.metaData() match { + // case Success(md) => md + // case _ => throw new Exception("target metadata error!") + // } + + // get data + val sourceData: RDD[(Product, Map[String, Any])] = sourceDataConnector.data() match { + case Success(dt) => dt + case Failure(ex) => throw ex + } + val targetData: RDD[(Product, Map[String, Any])] = targetDataConnector.data() match { + case Success(dt) => dt + case Failure(ex) => throw ex + } + + // accuracy algorithm + val (accuResult, missingRdd, matchedRdd) = accuracy(sourceData, targetData, ruleAnalyzer) + + // end time + val endTime = new Date().getTime + persist.log(endTime, s"calculation using time: ${endTime - startTime} ms") + + // persist result + persist.result(endTime, accuResult) + val missingRecords = missingRdd.map(record2String(_, ruleAnalyzer.sourceRuleExprs.persistExprs, ruleAnalyzer.targetRuleExprs.persistExprs)) + persist.missRecords(missingRecords) + + // persist end time + val persistEndTime = new Date().getTime + persist.log(persistEndTime, s"persist using time: ${persistEndTime - endTime} ms") + + // finish + persist.finish() + + // context stop + sc.stop + } + } + +} diff --git a/measure/src/main/scala/org/apache/griffin/measure/config/params/env/SparkParam.scala b/measure/src/main/scala/org/apache/griffin/measure/config/params/env/SparkParam.scala index c291ca79a..d6fb3647a 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/config/params/env/SparkParam.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/config/params/env/SparkParam.scala @@ -21,6 +21,7 @@ import org.apache.griffin.measure.config.params.Param @JsonInclude(Include.NON_NULL) case class SparkParam( @JsonProperty("log.level") logLevel: String, @JsonProperty("checkpoint.dir") cpDir: String, + @JsonProperty("batch.interval") batchInterval: String, @JsonProperty("config") config: Map[String, Any] ) extends Param { diff --git a/measure/src/main/scala/org/apache/griffin/measure/connector/DataConnectorFactory.scala b/measure/src/main/scala/org/apache/griffin/measure/connector/DataConnectorFactory.scala index 058ab8991..589da9f56 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/connector/DataConnectorFactory.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/connector/DataConnectorFactory.scala @@ -17,7 +17,6 @@ package org.apache.griffin.measure.connector import kafka.serializer.StringDecoder import org.apache.griffin.measure.config.params.user._ import org.apache.griffin.measure.rule.RuleExprs -import org.apache.griffin.measure.rule.expr._ import org.apache.spark.sql.SQLContext import org.apache.spark.streaming.StreamingContext import org.apache.spark.streaming.dstream.InputDStream @@ -31,6 +30,8 @@ object DataConnectorFactory { val HiveRegex = """^(?i)hive$""".r val AvroRegex = """^(?i)avro$""".r + val KafkaRegex = """^(?i)kafka$""".r + def getBatchDataConnector(sqlContext: SQLContext, dataConnectorParam: DataConnectorParam, ruleExprs: RuleExprs, @@ -58,32 +59,39 @@ object DataConnectorFactory { val config = dataConnectorParam.config Try { conType match { - case HiveRegex() => { - val KeyType = "key.type" - val ValueType = "value.type" - val keyType = dataConnectorParam.config.getOrElse(KeyType, "java.lang.String").toString - val valueType = dataConnectorParam.config.getOrElse(ValueType, "java.lang.String").toString - (getClassTag(keyType), getClassTag(valueType)) match { - case (ClassTag(k: Class[String]), ClassTag(v: Class[String])) => { - new KafkaDataConnector(ssc, config) { - type K = String - type KD = StringDecoder - type V = String - type VD = StringDecoder + case KafkaRegex() => { + genKafkaDataConnector(ssc, config) + } + case _ => throw new Exception("connector creation error!") + } + } + } + + protected def genKafkaDataConnector(ssc: StreamingContext, config: Map[String, Any]) = { + val KeyType = "key.type" + val ValueType = "value.type" + val keyType = config.getOrElse(KeyType, "java.lang.String").toString + val valueType = config.getOrElse(ValueType, "java.lang.String").toString + (getClassTag(keyType), getClassTag(valueType)) match { + case (ClassTag(k: Class[String]), ClassTag(v: Class[String])) => { + new KafkaDataConnector(ssc, config) { + type K = String + type KD = StringDecoder + type V = String + type VD = StringDecoder - def stream(): Try[InputDStream[(K, V)]] = Try { - val topicSet = topics.split(",").toSet - KafkaUtils.createDirectStream[K, V, KD, VD]( - ssc, - kafkaConfig, - topicSet - ) - } - } - } + def stream(): Try[InputDStream[(K, V)]] = Try { + val topicSet = topics.split(",").toSet + KafkaUtils.createDirectStream[K, V, KD, VD]( + ssc, + kafkaConfig, + topicSet + ) } } - case _ => throw new Exception("connector creation error!") + } + case _ => { + throw new Exception("not supported type kafka data connector") } } } diff --git a/measure/src/main/scala/org/apache/griffin/measure/connector/KafkaDataConnector.scala b/measure/src/main/scala/org/apache/griffin/measure/connector/KafkaDataConnector.scala index 5d9d0e40c..98644b3c7 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/connector/KafkaDataConnector.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/connector/KafkaDataConnector.scala @@ -1,13 +1,7 @@ package org.apache.griffin.measure.connector -import kafka.serializer.{Decoder, DefaultDecoder, StringDecoder} -import org.apache.griffin.measure.rule.RuleExprs +import kafka.serializer.Decoder import org.apache.spark.streaming.StreamingContext -import org.apache.spark.streaming.dstream.InputDStream -import org.apache.spark.streaming.kafka.KafkaUtils - -import scala.reflect.ClassTag -import scala.util.Try abstract class KafkaDataConnector(ssc: StreamingContext, config: Map[String, Any] ) extends StreamingDataConnector { @@ -26,44 +20,4 @@ abstract class KafkaDataConnector(ssc: StreamingContext, config: Map[String, Any def available(): Boolean = { true } -} - -object KafkaTool { - def generateKafkaDataConnector(ssc: StreamingContext, config: Map[String, Any]): KafkaDataConnector = { - val KeyType = "key.type" - val ValueType = "value.type" - - val keyType = config.getOrElse(KeyType, "java.lang.String").toString - val valueType = config.getOrElse(ValueType, "java.lang.String").toString - - (getClassTag(keyType), getClassTag(valueType)) match { - case (ClassTag(k: Class[String]), ClassTag(v: Class[String])) => { - new KafkaDataConnector(ssc, config) { - type K = String - type KD = StringDecoder - type V = String - type VD = StringDecoder - - def stream(): Try[InputDStream[(K, V)]] = Try { - val topicSet = topics.split(",").toSet - KafkaUtils.createDirectStream[K, V, KD, VD]( - ssc, - kafkaConfig, - topicSet - ) - } - } - } - } - - } - - private def getClassTag(tp: String): ClassTag[_] = { - try { - val clazz = Class.forName(tp) - ClassTag(clazz) - } catch { - case e: Throwable => throw e - } - } } \ No newline at end of file diff --git a/measure/src/main/scala/org/apache/griffin/measure/connector/KafkaStreamingDataConnector.scala b/measure/src/main/scala/org/apache/griffin/measure/connector/KafkaStreamingDataConnector.scala deleted file mode 100644 index 6b865bf0d..000000000 --- a/measure/src/main/scala/org/apache/griffin/measure/connector/KafkaStreamingDataConnector.scala +++ /dev/null @@ -1,89 +0,0 @@ -///*- -// * Licensed under the Apache License, Version 2.0 (the "License"); -//you may not use this file except in compliance with the License. -//You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -//Unless required by applicable law or agreed to in writing, software -//distributed under the License is distributed on an "AS IS" BASIS, -//WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -//See the License for the specific language governing permissions and -//limitations under the License. -// -// */ -//package org.apache.griffin.measure.connector -// -//import kafka.serializer._ -//import org.apache.griffin.measure.rule.RuleExprs -//import org.apache.spark.streaming.StreamingContext -//import org.apache.spark.streaming.dstream.InputDStream -//import org.apache.spark.streaming.kafka.KafkaUtils -// -//import scala.reflect.ClassTag -// -//import scala.util.Try -// -// -//case class KafkaStreamingDataConnector(ssc: StreamingContext, config: Map[String, Any], -// ruleExprs: RuleExprs, constFinalExprValueMap: Map[String, Any] -// ) extends StreamingDataConnector { -// -// val KafkaConfig = "kafka.config" -// val Topics = "topics" -// val KeyType = "key.type" -// val KeyDeserializer = "key.deserializer" -// val ValueType = "value.type" -// val ValueDeserializer = "value.deserializer" -// -// val kafkaConfig = config.get(KafkaConfig) match { -// case map: Map[String, Any] => map.mapValues(_.toString) -// case _ => Map[String, String]() -// } -// val topics = config.getOrElse(Topics, "").toString -// -// val keyType = config.getOrElse(KeyType, "java.lang.String").toString -// val valueType = config.getOrElse(ValueType, "java.lang.String").toString -// -//// import scala.reflect.runtime.universe._ -//// val m = runtimeMirror(getClass.getClassLoader) -//// val classSymbol = m.staticClass("java.lang.String") -//// val tpe = classSymbol.selfType -// -// def available(): Boolean = { -// true -// } -// -// def stream(): Try[InputDStream[(_, _)]] = Try { -// val topicSet = topics.split(",").toSet -// val keyClassTag = getClassTag(keyType) -// val valueClassTag = getClassTag(valueType) -// KafkaUtils.createDirectStream( -// ssc, -// kafkaConfig, -// topicSet -// )(keyClassTag, valueClassTag, getDeserializer(keyClassTag), getDeserializer(valueClassTag)) -// } -// -// private def getClassTag(tp: String): ClassTag[_] = { -// try { -// val clazz = Class.forName(tp) -// ClassTag(clazz) -// } catch { -// case e: Throwable => throw e -// } -// -// } -// -// private def getDeserializer[T](ct: ClassTag[T]): ClassTag[Decoder[T]] = { -// try { -// ct match { -// case ClassTag(clz: Class[String]) => ClassTag(classOf[StringDecoder]) -// case _ => ClassTag(classOf[DefaultDecoder]) -// } -// } catch { -// case e: Throwable => throw e -// } -// } -// -//} diff --git a/measure/src/main/scala/org/apache/griffin/measure/rule/expr/LiteralExpr.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/expr/LiteralExpr.scala index 1a4dcb57c..49e9d8284 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rule/expr/LiteralExpr.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rule/expr/LiteralExpr.scala @@ -14,6 +14,8 @@ limitations under the License. */ package org.apache.griffin.measure.rule.expr +import org.apache.griffin.measure.utils.TimeUtil + import scala.util.{Failure, Success, Try} trait LiteralExpr extends Expr { @@ -46,27 +48,7 @@ case class LiteralNumberExpr(expr: String) extends LiteralExpr { case class LiteralTimeExpr(expr: String) extends LiteralExpr { final val TimeRegex = """(\d+)(d|h|m|s|ms)""".r - val value: Option[Long] = { - Try { - expr match { - case TimeRegex(time, unit) => { - val t = time.toLong - unit match { - case "d" => t * 24 * 60 * 60 * 1000 - case "h" => t * 60 * 60 * 1000 - case "m" => t * 60 * 1000 - case "s" => t * 1000 - case "ms" => t - case _ => throw new Exception(s"${expr} is invalid time format") - } - } - case _ => throw new Exception(s"${expr} is invalid time format") - } - } match { - case Success(v) => Some(v) - case Failure(ex) => throw ex - } - } + val value: Option[Long] = TimeUtil.milliseconds(expr) val desc: String = expr } diff --git a/measure/src/main/scala/org/apache/griffin/measure/utils/TimeUtil.scala b/measure/src/main/scala/org/apache/griffin/measure/utils/TimeUtil.scala new file mode 100644 index 000000000..d235bd8b2 --- /dev/null +++ b/measure/src/main/scala/org/apache/griffin/measure/utils/TimeUtil.scala @@ -0,0 +1,47 @@ +/*- + * Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. + + */ +package org.apache.griffin.measure.utils + +import scala.util.{Failure, Success, Try} + +object TimeUtil { + + final val TimeRegex = """(\d+)(d|h|m|s|ms)""".r + + def milliseconds(timeString: String): Option[Long] = { + val value: Option[Long] = { + Try { + timeString match { + case TimeRegex(time, unit) => { + val t = time.toLong + unit match { + case "d" => t * 24 * 60 * 60 * 1000 + case "h" => t * 60 * 60 * 1000 + case "m" => t * 60 * 1000 + case "s" => t * 1000 + case "ms" => t + case _ => throw new Exception(s"${timeString} is invalid time format") + } + } + case _ => throw new Exception(s"${timeString} is invalid time format") + } + } match { + case Success(v) => Some(v) + case Failure(ex) => throw ex + } + } + } + +} From 3aea9317abef2875fa3dd50e8162a4c4720e96fb Mon Sep 17 00:00:00 2001 From: Lionel Liu Date: Wed, 5 Jul 2017 13:00:00 +0800 Subject: [PATCH 008/111] v3 --- measure/pom.xml | 8 + .../src/main/resources/config-streaming.json | 20 +- measure/src/main/resources/env.json | 14 ++ .../algo/batch/BatchAccuracyAlgo.scala | 7 +- .../measure/algo/batch/BatchProfileAlgo.scala | 7 +- .../measure/algo/core/AccuracyCore.scala | 4 +- .../streaming/StreamingAccuracyAlgo.scala | 179 ++++++++++---- .../griffin/measure/cache/CacheLock.scala | 27 +++ .../griffin/measure/cache/InfoCache.scala | 32 +++ .../measure/cache/InfoCacheFactory.scala | 23 ++ .../measure/cache/InfoCacheInstance.scala | 40 ++++ .../griffin/measure/cache/ZKCacheLock.scala | 45 ++++ .../griffin/measure/cache/ZKInfoCache.scala | 186 +++++++++++++++ .../measure/config/params/env/EnvParam.scala | 1 + .../config/params/env/InfoCacheParam.scala | 26 ++ .../params/user/DataConnectorParam.scala | 10 + .../connector/AvroBatchDataConnector.scala | 71 ++++-- .../connector/BatchDataConnector.scala | 2 +- .../connector/CacheDataConnector.scala | 40 ++++ .../measure/connector/DataConnector.scala | 6 +- .../connector/DataConnectorFactory.scala | 55 +++-- .../connector/HiveBatchDataConnector.scala | 72 ++++-- .../connector/KafkaDataConnector.scala | 219 ++++++++++++++++- .../KafkaStreamingDataConnector.scala | 49 ++++ .../connector/TempCacheDataConnector.scala | 63 +++++ .../measure/persist/PersistFactory.scala | 8 +- .../{ResultInfo.scala => DataInfo.scala} | 39 ++- .../rule/DataTypeCalculationUtil.scala | 114 +++++++++ .../griffin/measure/rule/ExprValueUtil.scala | 223 +++++++++++++++--- .../measure/rule/SchemaValueCombineUtil.scala | 196 +++++++++++++++ .../griffin/measure/rule/expr/Expr.scala | 4 + .../measure/rule/expr/LiteralExpr.scala | 7 + .../measure/rule/expr/LogicalExpr.scala | 2 + .../griffin/measure/rule/expr/MathExpr.scala | 15 ++ .../rule/func/DefaultFunctionDefine.scala | 29 +++ .../measure/rule/func/FunctionDefine.scala | 21 ++ .../measure/rule/func/FunctionUtil.scala | 70 ++++++ .../griffin/measure/utils/JsonUtil.scala | 4 + .../griffin/measure/utils/TimeUtil.scala | 23 ++ .../src/main/resources/sparkJob.properties | 2 +- 40 files changed, 1776 insertions(+), 187 deletions(-) create mode 100644 measure/src/main/scala/org/apache/griffin/measure/cache/CacheLock.scala create mode 100644 measure/src/main/scala/org/apache/griffin/measure/cache/InfoCache.scala create mode 100644 measure/src/main/scala/org/apache/griffin/measure/cache/InfoCacheFactory.scala create mode 100644 measure/src/main/scala/org/apache/griffin/measure/cache/InfoCacheInstance.scala create mode 100644 measure/src/main/scala/org/apache/griffin/measure/cache/ZKCacheLock.scala create mode 100644 measure/src/main/scala/org/apache/griffin/measure/cache/ZKInfoCache.scala create mode 100644 measure/src/main/scala/org/apache/griffin/measure/config/params/env/InfoCacheParam.scala create mode 100644 measure/src/main/scala/org/apache/griffin/measure/connector/CacheDataConnector.scala create mode 100644 measure/src/main/scala/org/apache/griffin/measure/connector/KafkaStreamingDataConnector.scala create mode 100644 measure/src/main/scala/org/apache/griffin/measure/connector/TempCacheDataConnector.scala rename measure/src/main/scala/org/apache/griffin/measure/result/{ResultInfo.scala => DataInfo.scala} (54%) create mode 100644 measure/src/main/scala/org/apache/griffin/measure/rule/DataTypeCalculationUtil.scala create mode 100644 measure/src/main/scala/org/apache/griffin/measure/rule/SchemaValueCombineUtil.scala create mode 100644 measure/src/main/scala/org/apache/griffin/measure/rule/func/DefaultFunctionDefine.scala create mode 100644 measure/src/main/scala/org/apache/griffin/measure/rule/func/FunctionDefine.scala create mode 100644 measure/src/main/scala/org/apache/griffin/measure/rule/func/FunctionUtil.scala diff --git a/measure/pom.xml b/measure/pom.xml index 444c09068..f337ffe24 100644 --- a/measure/pom.xml +++ b/measure/pom.xml @@ -45,6 +45,7 @@ 3.0.0 1.7.21 1.2.16 + 2.10.0 3.6.0 @@ -135,6 +136,13 @@ ${log4j.version} + + + org.apache.curator + curator-recipes + ${curator.version} + + junit diff --git a/measure/src/main/resources/config-streaming.json b/measure/src/main/resources/config-streaming.json index e5a83dde3..4a355486b 100644 --- a/measure/src/main/resources/config-streaming.json +++ b/measure/src/main/resources/config-streaming.json @@ -14,7 +14,14 @@ }, "topics": "src", "key.type": "java.lang.String", - "value.type": "java.lang.String" + "value.type": "java.lang.String", + "cache": { + "type": "temp", + "config": { + "table.name": "source", + "info.path": "src" + } + } } }, @@ -30,7 +37,16 @@ }, "topics": "tgt", "key.type": "java.lang.String", - "value.type": "java.lang.String" + "value.type": "java.lang.String", + "cache": { + "type": "hive", + "version": 1.2, + "config": { + "database": "default", + "table.name": "target_table", + "info.path": "tgt" + } + } } }, diff --git a/measure/src/main/resources/env.json b/measure/src/main/resources/env.json index 31acc7b6e..10250feb6 100644 --- a/measure/src/main/resources/env.json +++ b/measure/src/main/resources/env.json @@ -24,6 +24,20 @@ } ], + "info.cache": [ + { + "type": "zk", + "config": { + "hosts": "localhost:2181", + "parent.path": "/griffin/cache", + "mode": "persist", + "init.clear": true, + "close.clear": false, + "lock.path": "lock" + } + } + ], + "cleaner": { } diff --git a/measure/src/main/scala/org/apache/griffin/measure/algo/batch/BatchAccuracyAlgo.scala b/measure/src/main/scala/org/apache/griffin/measure/algo/batch/BatchAccuracyAlgo.scala index a29a5f14c..deced368e 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/algo/batch/BatchAccuracyAlgo.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/algo/batch/BatchAccuracyAlgo.scala @@ -39,8 +39,11 @@ case class BatchAccuracyAlgo(allParam: AllParam) extends AccuracyAlgo { Try { val metricName = userParam.name + val sparkParam = envParam.sparkParam + val conf = new SparkConf().setAppName(metricName) val sc = new SparkContext(conf) + sc.setLogLevel(sparkParam.logLevel) val sqlContext = new HiveContext(sc) // start time @@ -61,8 +64,8 @@ case class BatchAccuracyAlgo(allParam: AllParam) extends AccuracyAlgo { val ruleAnalyzer: RuleAnalyzer = RuleAnalyzer(rule) // const expr value map - val constExprValueMap = ExprValueUtil.genExprValueMap(None, ruleAnalyzer.constCacheExprs, Map[String, Any]()) - val finalConstExprValueMap = ExprValueUtil.updateExprValueMap(ruleAnalyzer.constFinalCacheExprs, constExprValueMap) + val constExprValueMap = ExprValueUtil.genExprValueMaps(None, ruleAnalyzer.constCacheExprs, Map[String, Any]()) + val finalConstExprValueMap = ExprValueUtil.updateExprValueMaps(ruleAnalyzer.constFinalCacheExprs, constExprValueMap) // data connector val sourceDataConnector: BatchDataConnector = diff --git a/measure/src/main/scala/org/apache/griffin/measure/algo/batch/BatchProfileAlgo.scala b/measure/src/main/scala/org/apache/griffin/measure/algo/batch/BatchProfileAlgo.scala index c7b33c583..21af67a40 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/algo/batch/BatchProfileAlgo.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/algo/batch/BatchProfileAlgo.scala @@ -39,8 +39,11 @@ case class BatchProfileAlgo(allParam: AllParam) extends ProfileAlgo { Try { val metricName = userParam.name + val sparkParam = envParam.sparkParam + val conf = new SparkConf().setAppName(metricName) val sc = new SparkContext(conf) + sc.setLogLevel(sparkParam.logLevel) val sqlContext = new HiveContext(sc) // start time @@ -61,8 +64,8 @@ case class BatchProfileAlgo(allParam: AllParam) extends ProfileAlgo { val ruleAnalyzer: RuleAnalyzer = RuleAnalyzer(rule) // const expr value map - val constExprValueMap = ExprValueUtil.genExprValueMap(None, ruleAnalyzer.constCacheExprs, Map[String, Any]()) - val finalConstExprValueMap = ExprValueUtil.updateExprValueMap(ruleAnalyzer.constFinalCacheExprs, constExprValueMap) + val constExprValueMap = ExprValueUtil.genExprValueMaps(None, ruleAnalyzer.constCacheExprs, Map[String, Any]()) + val finalConstExprValueMap = ExprValueUtil.updateExprValueMaps(ruleAnalyzer.constFinalCacheExprs, constExprValueMap) // data connector val sourceDataConnector: BatchDataConnector = diff --git a/measure/src/main/scala/org/apache/griffin/measure/algo/core/AccuracyCore.scala b/measure/src/main/scala/org/apache/griffin/measure/algo/core/AccuracyCore.scala index cbea053b1..d7d45d0a7 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/algo/core/AccuracyCore.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/algo/core/AccuracyCore.scala @@ -83,9 +83,9 @@ object AccuracyCore { } // currently we can not get the mismatch reason, we need to add such information to figure out how it mismatches if (matched) (matched, Map[String, Any]()) - else (matched, Map[String, Any](MismatchInfo.wrap("not matched"), TargetInfo.wrap(target._1))) + else (matched, Map[String, Any](MismatchInfo.wrap("not matched"))) } else { - (false, Map[String, Any](MismatchInfo.wrap("invalid to compare"), TargetInfo.wrap(target._1))) + (false, Map[String, Any](MismatchInfo.wrap("invalid to compare"))) } } diff --git a/measure/src/main/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgo.scala b/measure/src/main/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgo.scala index 9d5bd4de1..5ed81c63e 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgo.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgo.scala @@ -15,14 +15,19 @@ limitations under the License. package org.apache.griffin.measure.algo.streaming import java.util.Date +import java.util.concurrent.TimeUnit import org.apache.griffin.measure.algo.AccuracyAlgo +import org.apache.griffin.measure.algo.core.AccuracyCore +import org.apache.griffin.measure.cache.InfoCacheInstance import org.apache.griffin.measure.config.params.AllParam -import org.apache.griffin.measure.connector.{DataConnector, DataConnectorFactory} +import org.apache.griffin.measure.connector._ import org.apache.griffin.measure.persist.{Persist, PersistFactory} +import org.apache.griffin.measure.result.AccuracyResult import org.apache.griffin.measure.rule.{ExprValueUtil, RuleAnalyzer, RuleFactory} -import org.apache.griffin.measure.rule.expr.StatementExpr +import org.apache.griffin.measure.rule.expr.{Expr, StatementExpr} import org.apache.griffin.measure.utils.TimeUtil +import org.apache.spark.rdd.RDD import org.apache.spark.sql.hive.HiveContext import org.apache.spark.streaming.{Milliseconds, Seconds, StreamingContext} import org.apache.spark.{SparkConf, SparkContext} @@ -38,16 +43,19 @@ case class StreamingAccuracyAlgo(allParam: AllParam) extends AccuracyAlgo { Try { val metricName = userParam.name + val sparkParam = envParam.sparkParam + val conf = new SparkConf().setAppName(metricName) val sc = new SparkContext(conf) + sc.setLogLevel(sparkParam.logLevel) val sqlContext = new HiveContext(sc) - val interval = TimeUtil.milliseconds(envParam.sparkParam.batchInterval) match { + val interval = TimeUtil.milliseconds(sparkParam.batchInterval) match { case Some(interval) => Milliseconds(interval) case _ => throw new Exception("invalid batch interval") } val ssc = new StreamingContext(sc, interval) - ssc.checkpoint(envParam.sparkParam.cpDir) + ssc.checkpoint(sparkParam.cpDir) // start time val startTime = new Date().getTime() @@ -67,13 +75,13 @@ case class StreamingAccuracyAlgo(allParam: AllParam) extends AccuracyAlgo { val ruleAnalyzer: RuleAnalyzer = RuleAnalyzer(rule) // const expr value map - val constExprValueMap = ExprValueUtil.genExprValueMap(None, ruleAnalyzer.constCacheExprs, Map[String, Any]()) - val finalConstExprValueMap = ExprValueUtil.updateExprValueMap(ruleAnalyzer.constFinalCacheExprs, constExprValueMap) + val constExprValueMap = ExprValueUtil.genExprValueMaps(None, ruleAnalyzer.constCacheExprs, Map[String, Any]()) + val finalConstExprValueMap = ExprValueUtil.updateExprValueMaps(ruleAnalyzer.constFinalCacheExprs, constExprValueMap) // data connector - val sourceDataConnector: DataConnector = - DataConnectorFactory.getStreamingDataConnector(ssc, userParam.sourceParam, - ruleAnalyzer.sourceRuleExprs, finalConstExprValueMap + val sourceDataConnector: BatchDataConnector = + DataConnectorFactory.getDataConnector(sqlContext, ssc, userParam.sourceParam, + ruleAnalyzer.sourceRuleExprs, finalConstExprValueMap.head ) match { case Success(cntr) => { if (cntr.available) cntr @@ -81,9 +89,9 @@ case class StreamingAccuracyAlgo(allParam: AllParam) extends AccuracyAlgo { } case Failure(ex) => throw ex } - val targetDataConnector: DataConnector = - DataConnectorFactory.getStreamingDataConnector(sqlContext, userParam.targetParam, - ruleAnalyzer.targetRuleExprs, finalConstExprValueMap + val targetDataConnector: BatchDataConnector = + DataConnectorFactory.getDataConnector(sqlContext, ssc, userParam.targetParam, + ruleAnalyzer.targetRuleExprs, finalConstExprValueMap.head ) match { case Success(cntr) => { if (cntr.available) cntr @@ -92,47 +100,128 @@ case class StreamingAccuracyAlgo(allParam: AllParam) extends AccuracyAlgo { case Failure(ex) => throw ex } - // get metadata - // val sourceMetaData: Iterable[(String, String)] = sourceDataConnector.metaData() match { - // case Success(md) => md - // case _ => throw new Exception("source metadata error!") - // } - // val targetMetaData: Iterable[(String, String)] = targetDataConnector.metaData() match { - // case Success(md) => md - // case _ => throw new Exception("target metadata error!") - // } - - // get data - val sourceData: RDD[(Product, Map[String, Any])] = sourceDataConnector.data() match { - case Success(dt) => dt - case Failure(ex) => throw ex - } - val targetData: RDD[(Product, Map[String, Any])] = targetDataConnector.data() match { - case Success(dt) => dt - case Failure(ex) => throw ex + InfoCacheInstance.initInstance(envParam.infoCacheParams, metricName) + InfoCacheInstance.init + + // process thread + case class Process() extends Runnable { + val lock = InfoCacheInstance.genLock + def run(): Unit = { + val locked = lock.lock(5, TimeUnit.SECONDS) + if (locked) { + try { + val st = new Date().getTime + // get data + val sourceData = sourceDataConnector.data match { + case Success(dt) => dt + case Failure(ex) => throw ex + } + val targetData = targetDataConnector.data match { + case Success(dt) => dt + case Failure(ex) => throw ex + } + + // accuracy algorithm + val (accuResult, missingRdd, matchedRdd) = accuracy(sourceData, targetData, ruleAnalyzer) + + val et = new Date().getTime + + // persist time + persist.log(et, s"calculation using time: ${et - st} ms") + + // persist result + persist.result(et, accuResult) + val missingRecords = missingRdd.map(record2String(_, ruleAnalyzer.sourceRuleExprs.persistExprs, ruleAnalyzer.targetRuleExprs.persistExprs)) + persist.missRecords(missingRecords) + + val pet = new Date().getTime + persist.log(pet, s"persist using time: ${pet - et} ms") + } finally { + lock.unlock() + } + } + } } - // accuracy algorithm - val (accuResult, missingRdd, matchedRdd) = accuracy(sourceData, targetData, ruleAnalyzer) + // fixme: thread pool - // end time - val endTime = new Date().getTime - persist.log(endTime, s"calculation using time: ${endTime - startTime} ms") + // get data +// val sourceData: RDD[(Product, Map[String, Any])] = sourceDataConnector.data() match { +// case Success(dt) => dt +// case Failure(ex) => throw ex +// } +// val targetData: RDD[(Product, Map[String, Any])] = targetDataConnector.data() match { +// case Success(dt) => dt +// case Failure(ex) => throw ex +// } +// +// // accuracy algorithm +// val (accuResult, missingRdd, matchedRdd) = accuracy(sourceData, targetData, ruleAnalyzer) +// +// // end time +// val endTime = new Date().getTime +// persist.log(endTime, s"calculation using time: ${endTime - startTime} ms") +// +// // persist result +// persist.result(endTime, accuResult) +// val missingRecords = missingRdd.map(record2String(_, ruleAnalyzer.sourceRuleExprs.persistExprs, ruleAnalyzer.targetRuleExprs.persistExprs)) +// persist.missRecords(missingRecords) +// +// // persist end time +// val persistEndTime = new Date().getTime +// persist.log(persistEndTime, s"persist using time: ${persistEndTime - endTime} ms") +// +// // finish +// persist.finish() + + ssc.start() + ssc.awaitTermination() + ssc.stop(stopSparkContext=true, stopGracefully=true) - // persist result - persist.result(endTime, accuResult) - val missingRecords = missingRdd.map(record2String(_, ruleAnalyzer.sourceRuleExprs.persistExprs, ruleAnalyzer.targetRuleExprs.persistExprs)) - persist.missRecords(missingRecords) + // context stop + sc.stop - // persist end time - val persistEndTime = new Date().getTime - persist.log(persistEndTime, s"persist using time: ${persistEndTime - endTime} ms") + InfoCacheInstance.close - // finish persist.finish() + } + } - // context stop - sc.stop + // calculate accuracy between source data and target data + def accuracy(sourceData: RDD[(Product, (Map[String, Any], Map[String, Any]))], + targetData: RDD[(Product, (Map[String, Any], Map[String, Any]))], + ruleAnalyzer: RuleAnalyzer) = { + // 1. cogroup + val allKvs = sourceData.cogroup(targetData) + + // 2. accuracy calculation + val (accuResult, missingRdd, matchedRdd) = AccuracyCore.accuracy(allKvs, ruleAnalyzer) + + (accuResult, missingRdd, matchedRdd) + } + + // convert data into a string + def record2String(rec: (Product, (Map[String, Any], Map[String, Any])), sourcePersist: Iterable[Expr], targetPersist: Iterable[Expr]): String = { + val (key, (data, info)) = rec + val persistData = getPersistMap(data, sourcePersist) + val persistInfo = info.mapValues { value => + value match { + case vd: Map[String, Any] => getPersistMap(vd, targetPersist) + case v => v + } + } + s"${persistData} [${persistInfo}]" + } + + // get the expr value map of the persist expressions + private def getPersistMap(data: Map[String, Any], persist: Iterable[Expr]): Map[String, Any] = { + val persistMap = persist.map(e => (e._id, e.desc)).toMap + data.flatMap { pair => + val (k, v) = pair + persistMap.get(k) match { + case Some(d) => Some((d -> v)) + case _ => None + } } } diff --git a/measure/src/main/scala/org/apache/griffin/measure/cache/CacheLock.scala b/measure/src/main/scala/org/apache/griffin/measure/cache/CacheLock.scala new file mode 100644 index 000000000..623be23e4 --- /dev/null +++ b/measure/src/main/scala/org/apache/griffin/measure/cache/CacheLock.scala @@ -0,0 +1,27 @@ +/*- + * Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. + + */ +package org.apache.griffin.measure.cache + +import java.util.concurrent.TimeUnit + +import org.apache.griffin.measure.log.Loggable + +trait CacheLock extends Loggable with Serializable { + + def lock(outtime: Long, unit: TimeUnit): Boolean + + def unlock(): Unit + +} diff --git a/measure/src/main/scala/org/apache/griffin/measure/cache/InfoCache.scala b/measure/src/main/scala/org/apache/griffin/measure/cache/InfoCache.scala new file mode 100644 index 000000000..72496b8ba --- /dev/null +++ b/measure/src/main/scala/org/apache/griffin/measure/cache/InfoCache.scala @@ -0,0 +1,32 @@ +/*- + * Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. + + */ +package org.apache.griffin.measure.cache + +import org.apache.griffin.measure.log.Loggable + +trait InfoCache extends Loggable with Serializable { + + def init(): Unit + def available(): Boolean + def close(): Unit + + def cacheInfo(info: Map[String, String]): Boolean + def readInfo(keys: Iterable[String]): Map[String, String] + def deleteInfo(keys: Iterable[String]): Unit + def clearInfo(): Unit + + def genLock(): CacheLock + +} diff --git a/measure/src/main/scala/org/apache/griffin/measure/cache/InfoCacheFactory.scala b/measure/src/main/scala/org/apache/griffin/measure/cache/InfoCacheFactory.scala new file mode 100644 index 000000000..9cce29dde --- /dev/null +++ b/measure/src/main/scala/org/apache/griffin/measure/cache/InfoCacheFactory.scala @@ -0,0 +1,23 @@ +package org.apache.griffin.measure.cache + +import org.apache.griffin.measure.config.params.env.InfoCacheParam + +import scala.util.{Success, Try} + +case class InfoCacheFactory(infoCacheParams: Iterable[InfoCacheParam], metricName: String) extends Serializable { + + val ZK_REGEX = """^(?i)zk|zookeeper$""".r + + def getInfoCache(infoCacheParam: InfoCacheParam): Option[InfoCache] = { + val config = infoCacheParam.config + val infoCacheTry = infoCacheParam.persistType match { + case ZK_REGEX() => Try(ZKInfoCache(config, metricName)) + case _ => throw new Exception("not supported info cache type") + } + infoCacheTry match { + case Success(infoCache) if (infoCache.available) => Some(infoCache) + case _ => None + } + } + +} \ No newline at end of file diff --git a/measure/src/main/scala/org/apache/griffin/measure/cache/InfoCacheInstance.scala b/measure/src/main/scala/org/apache/griffin/measure/cache/InfoCacheInstance.scala new file mode 100644 index 000000000..65ae24889 --- /dev/null +++ b/measure/src/main/scala/org/apache/griffin/measure/cache/InfoCacheInstance.scala @@ -0,0 +1,40 @@ +/*- + * Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. + + */ +package org.apache.griffin.measure.cache + +import org.apache.griffin.measure.config.params.env.InfoCacheParam + +object InfoCacheInstance extends InfoCache { + var infoCaches: List[InfoCache] = Nil + + def initInstance(infoCacheParams: Iterable[InfoCacheParam], metricName: String) = { + val fac = InfoCacheFactory(infoCacheParams, metricName) + infoCaches = infoCacheParams.flatMap(param => fac.getInfoCache(param)).toList + } + + def init(): Unit = infoCaches.foreach(_.init) + def available(): Boolean = infoCaches.foldLeft(false)(_ || _.available) + def close(): Unit = infoCaches.foreach(_.close) + + def cacheInfo(info: Map[String, String]): Boolean = { + infoCaches.foldLeft(false) { (res, infoCache) => res || infoCache.cacheInfo(info) } + } + def readInfo(keys: Iterable[String]): Map[String, String] = { + val maps = infoCaches.map(_.readInfo(keys)).reverse + maps.fold(Map[String, String]())(_ ++ _) + } + def deleteInfo(keys: Iterable[String]): Unit = infoCaches.foreach(_.deleteInfo(keys)) + def clearInfo(): Unit = infoCaches.foreach(_.clearInfo) +} \ No newline at end of file diff --git a/measure/src/main/scala/org/apache/griffin/measure/cache/ZKCacheLock.scala b/measure/src/main/scala/org/apache/griffin/measure/cache/ZKCacheLock.scala new file mode 100644 index 000000000..ccd732b85 --- /dev/null +++ b/measure/src/main/scala/org/apache/griffin/measure/cache/ZKCacheLock.scala @@ -0,0 +1,45 @@ +/*- + * Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. + + */ +package org.apache.griffin.measure.cache + +import java.util.concurrent.TimeUnit + +import org.apache.curator.framework.recipes.locks.InterProcessMutex + +case class ZKCacheLock(@transient mutex: InterProcessMutex) extends CacheLock { + + def lock(outtime: Long, unit: TimeUnit): Boolean = { + try { + mutex.acquire(outtime, unit) + } catch { + case e: Throwable => { + error(s"lock error: ${e.getMessage}") + false + } + } + + } + + def unlock(): Unit = { + try { + mutex.release + } catch { + case e: Throwable => { + error(s"unlock error: ${e.getMessage}") + } + } + } + +} diff --git a/measure/src/main/scala/org/apache/griffin/measure/cache/ZKInfoCache.scala b/measure/src/main/scala/org/apache/griffin/measure/cache/ZKInfoCache.scala new file mode 100644 index 000000000..9d7886869 --- /dev/null +++ b/measure/src/main/scala/org/apache/griffin/measure/cache/ZKInfoCache.scala @@ -0,0 +1,186 @@ +/*- + * Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. + + */ +package org.apache.griffin.measure.cache + +import org.apache.curator.framework.imps.CuratorFrameworkState +import org.apache.curator.framework.recipes.locks.InterProcessMutex +import org.apache.curator.framework.{CuratorFramework, CuratorFrameworkFactory} +import org.apache.curator.retry.ExponentialBackoffRetry +import org.apache.zookeeper.CreateMode + +case class ZKInfoCache(config: Map[String, Any], metricName: String) extends InfoCache { + + val Hosts = "hosts" + val ParentPath = "parent.path" + val Mode = "mode" + val InitClear = "init.clear" + val CloseClear = "close.clear" + val LockPath = "lock.path" + + val PersistRegex = """^(?i)persist$""".r + val EphemeralRegex = """^(?i)ephemeral$""".r + + final val separator = "/" + + val hosts = config.getOrElse(Hosts, "").toString + val parentPath = config.getOrElse(ParentPath, separator).toString + val mode: CreateMode = config.get(Mode) match { + case Some(s: String) => s match { + case PersistRegex() => CreateMode.PERSISTENT + case EphemeralRegex() => CreateMode.EPHEMERAL + case _ => CreateMode.PERSISTENT + } + case _ => CreateMode.PERSISTENT + } + val initClear = config.get(InitClear) match { + case Some(b: Boolean) => b + case _ => true + } + val closeClear = config.get(CloseClear) match { + case Some(b: Boolean) => b + case _ => false + } + val lockPath = config.getOrElse(LockPath, "lock").toString + + private val client: CuratorFramework = CuratorFrameworkFactory.newClient( + Hosts, new ExponentialBackoffRetry(1000, 3)).usingNamespace(cacheParentPath) + + def init(): Unit = { + client.start() + if (initClear) { + clearInfo + } + } + + def available(): Boolean = { + client.getState match { + case CuratorFrameworkState.STARTED => true + case _ => false + } + } + + def close(): Unit = { + if (closeClear) { + clearInfo + } + client.close() + } + + def cacheInfo(info: Map[String, String]): Boolean = { + info.foldLeft(true) { (rs, pair) => + val (k, v) = pair + createOrUpdate(path(k), v) && rs + } + } + + def readInfo(keys: Iterable[String]): Map[String, String] = { + keys.flatMap { key => + read(key) match { + case Some(v) => Some((key, v)) + case _ => None + } + }.toMap + } + + def deleteInfo(keys: Iterable[String]): Unit = { + keys.foreach { key => delete(path(key)) } + } + + def clearInfo(): Unit = { + delete("/") + } + + def genLock(s: String): ZKCacheLock = { + val lpt = if (s.isEmpty) path(lockPath) else validHeadPath(lockPath) + s + ZKCacheLock(new InterProcessMutex(client, lpt)) + } + + private def path(k: String): String = { + if (k.startsWith(separator)) k else separator + k + } + + private def validHeadPath(head: String): String = { + val hd = if (head.startsWith(separator)) head else separator + head + if (hd.endsWith(separator)) hd else hd + separator + } + + private def cacheParentPath: String = { + validHeadPath(parentPath) + metricName + } + + private def createOrUpdate(path: String, content: String): Boolean = { + if (checkExists(path)) { + update(path, content) + } else { + create(path, content) + } + } + + private def create(path: String, content: String): Boolean = { + try { + client.create().creatingParentsIfNeeded().withMode(mode) + .forPath(path, content.getBytes("utf-8")) + true + } catch { + case e: Throwable => { + error(s"create ( ${path} -> ${content} ) error: ${e.getMessage}") + false + } + } + } + + private def update(path: String, content: String): Boolean = { + try { + client.setData().forPath(path, content.getBytes("utf-8")) + true + } catch { + case e: Throwable => { + error(s"update ( ${path} -> ${content} ) error: ${e.getMessage}") + false + } + } + } + + private def read(path: String): Option[String] = { + try { + Some(new String(client.getData().forPath(path), "utf-8")) + } catch { + case e: Throwable => { + error(s"read ${path} error: ${e.getMessage}") + None + } + } + } + + private def delete(path: String): Unit = { + try { + client.delete().guaranteed().deletingChildrenIfNeeded().forPath(path) + } catch { + case e: Throwable => error(s"delete ${path} error: ${e.getMessage}") + } + } + + private def checkExists(path: String): Boolean = { + try { + client.checkExists().forPath(path) != null + } catch { + case e: Throwable => { + error(s"check exists ${path} error: ${e.getMessage}") + false + } + } + } + +} diff --git a/measure/src/main/scala/org/apache/griffin/measure/config/params/env/EnvParam.scala b/measure/src/main/scala/org/apache/griffin/measure/config/params/env/EnvParam.scala index ccef5627b..cab779536 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/config/params/env/EnvParam.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/config/params/env/EnvParam.scala @@ -21,6 +21,7 @@ import org.apache.griffin.measure.config.params.Param @JsonInclude(Include.NON_NULL) case class EnvParam( @JsonProperty("spark") sparkParam: SparkParam, @JsonProperty("persist") persistParams: List[PersistParam], + @JsonProperty("info.cache") infoCacheParams: List[InfoCacheParam], @JsonProperty("cleaner") cleanerParam: CleanerParam ) extends Param { diff --git a/measure/src/main/scala/org/apache/griffin/measure/config/params/env/InfoCacheParam.scala b/measure/src/main/scala/org/apache/griffin/measure/config/params/env/InfoCacheParam.scala new file mode 100644 index 000000000..6ddade9d0 --- /dev/null +++ b/measure/src/main/scala/org/apache/griffin/measure/config/params/env/InfoCacheParam.scala @@ -0,0 +1,26 @@ +/*- + * Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. + + */ +package org.apache.griffin.measure.config.params.env + +import com.fasterxml.jackson.annotation.{JsonInclude, JsonProperty} +import com.fasterxml.jackson.annotation.JsonInclude.Include +import org.apache.griffin.measure.config.params.Param + +@JsonInclude(Include.NON_NULL) +case class InfoCacheParam( @JsonProperty("type") persistType: String, + @JsonProperty("config") config: Map[String, Any] + ) extends Param { + +} diff --git a/measure/src/main/scala/org/apache/griffin/measure/config/params/user/DataConnectorParam.scala b/measure/src/main/scala/org/apache/griffin/measure/config/params/user/DataConnectorParam.scala index 74e104c31..56aa2a1ff 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/config/params/user/DataConnectorParam.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/config/params/user/DataConnectorParam.scala @@ -25,3 +25,13 @@ case class DataConnectorParam( @JsonProperty("type") conType: String, ) extends Param { } + +object DataConnectorParam { + def apply(map: Map[String, Any]): DataConnectorParam = { + DataConnectorParam( + map.getOrElse("type", "").toString, + map.getOrElse("version", "").toString, + map.getOrElse("config", Map[String, Any]()).asInstanceOf[Map[String, Any]] + ) + } +} \ No newline at end of file diff --git a/measure/src/main/scala/org/apache/griffin/measure/connector/AvroBatchDataConnector.scala b/measure/src/main/scala/org/apache/griffin/measure/connector/AvroBatchDataConnector.scala index 4f4f94112..2736d270b 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/connector/AvroBatchDataConnector.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/connector/AvroBatchDataConnector.scala @@ -22,6 +22,7 @@ import com.databricks.spark.avro._ import scala.util.{Success, Try} import java.nio.file.{Files, Paths} +import org.apache.griffin.measure.result._ import org.apache.griffin.measure.rule.{ExprValueUtil, RuleExprs} import org.apache.griffin.measure.utils.HdfsUtil @@ -57,36 +58,60 @@ case class AvroBatchDataConnector(sqlContext: SQLContext, config: Map[String, An } } - def data(): Try[RDD[(Product, Map[String, Any])]] = { + def data(): Try[RDD[(Product, (Map[String, Any], Map[String, Any]))]] = { Try { loadDataFile.flatMap { row => // generate cache data - val cacheExprValueMap: Map[String, Any] = ruleExprs.cacheExprs.foldLeft(constFinalExprValueMap) { (cachedMap, expr) => - ExprValueUtil.genExprValueMap(Some(row), expr, cachedMap) - } - val finalExprValueMap = ExprValueUtil.updateExprValueMap(ruleExprs.finalCacheExprs, cacheExprValueMap) - - // when clause filter data source - val whenResult = ruleExprs.whenClauseExprOpt match { - case Some(whenClause) => whenClause.calculate(finalExprValueMap) - case _ => None - } + val cacheExprValueMaps = ExprValueUtil.genExprValueMaps(Some(row), ruleExprs.cacheExprs, constFinalExprValueMap) + val finalExprValueMaps = ExprValueUtil.updateExprValueMaps(ruleExprs.finalCacheExprs, cacheExprValueMaps) + + // data info + val dataInfoMap: Map[String, Any] = DataInfo.cacheInfoList.map { info => + try { + (info.key -> row.getAs[info.T](info.key)) + } catch { + case e: Throwable => info.defWrap + } + }.toMap - // get groupby data - whenResult match { - case Some(false) => None - case _ => { - val groupbyData: Seq[AnyRef] = ruleExprs.groupbyExprs.flatMap { expr => - expr.calculate(finalExprValueMap) match { - case Some(v) => Some(v.asInstanceOf[AnyRef]) - case _ => None - } + finalExprValueMaps.flatMap { finalExprValueMap => + val groupbyData: Seq[AnyRef] = ruleExprs.groupbyExprs.flatMap { expr => + expr.calculate(finalExprValueMap) match { + case Some(v) => Some(v.asInstanceOf[AnyRef]) + case _ => None } - val key = toTuple(groupbyData) - - Some((key, finalExprValueMap)) } + val key = toTuple(groupbyData) + + Some((key, (finalExprValueMap, dataInfoMap))) } + +// val cacheExprValueMap: Map[String, Any] = ruleExprs.cacheExprs.foldLeft(constFinalExprValueMap) { (cachedMap, expr) => +// ExprValueUtil.genExprValueMaps(Some(row), expr, cachedMap) +// } +// val finalExprValueMap = ExprValueUtil.updateExprValueMaps(ruleExprs.finalCacheExprs, cacheExprValueMap) + + // when clause filter data source +// val whenResult = ruleExprs.whenClauseExprOpt match { +// case Some(whenClause) => whenClause.calculate(finalExprValueMap) +// case _ => None +// } +// +// // get groupby data +// whenResult match { +// case Some(false) => None +// case _ => { +// val groupbyData: Seq[AnyRef] = ruleExprs.groupbyExprs.flatMap { expr => +// expr.calculate(finalExprValueMap) match { +// case Some(v) => Some(v.asInstanceOf[AnyRef]) +// case _ => None +// } +// } +// val key = toTuple(groupbyData) +// +// Some((key, finalExprValueMap)) +// } +// } } } } diff --git a/measure/src/main/scala/org/apache/griffin/measure/connector/BatchDataConnector.scala b/measure/src/main/scala/org/apache/griffin/measure/connector/BatchDataConnector.scala index 12a093c8a..9e8b1e02d 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/connector/BatchDataConnector.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/connector/BatchDataConnector.scala @@ -23,6 +23,6 @@ trait BatchDataConnector extends DataConnector { def metaData(): Try[Iterable[(String, String)]] - def data(): Try[RDD[(Product, Map[String, Any])]] + def data(): Try[RDD[(Product, (Map[String, Any], Map[String, Any]))]] } diff --git a/measure/src/main/scala/org/apache/griffin/measure/connector/CacheDataConnector.scala b/measure/src/main/scala/org/apache/griffin/measure/connector/CacheDataConnector.scala new file mode 100644 index 000000000..2f8b8166c --- /dev/null +++ b/measure/src/main/scala/org/apache/griffin/measure/connector/CacheDataConnector.scala @@ -0,0 +1,40 @@ +package org.apache.griffin.measure.connector + +import org.apache.griffin.measure.cache.{InfoCacheInstance, ZKInfoCache} +import org.apache.spark.sql.DataFrame + +trait CacheDataConnector extends DataConnector { + + def saveData(df: DataFrame, ms: Long): Unit + + def readData(): DataFrame + + val cacheTimeKey: String + + val LastProcTime = "last.proc.time" + val CurReadyTime = "cur.ready.time" + + protected def submitCacheTime(ms: Long): Unit = { + val map = Map[String, String]() + (cacheTimeKey -> ms.toString) + InfoCacheInstance.cacheInfo(map) + } + + protected def readTimeRange(): (Long, Long) = { + val map = InfoCacheInstance.readInfo(List(LastProcTime, CurReadyTime)) + val lastProcTime = getLong(map, LastProcTime) + val curReadyTime = getLong(map, CurReadyTime) + (lastProcTime + 1, curReadyTime) + } + + private def getLong(map: Map[String, String], key: String): Long = { + try { + map.get(key) match { + case Some(v) => v.toLong + case _ => -1 + } + } catch { + case _ => -1 + } + } + +} diff --git a/measure/src/main/scala/org/apache/griffin/measure/connector/DataConnector.scala b/measure/src/main/scala/org/apache/griffin/measure/connector/DataConnector.scala index 5b6fefa06..2ac32499f 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/connector/DataConnector.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/connector/DataConnector.scala @@ -14,9 +14,13 @@ limitations under the License. */ package org.apache.griffin.measure.connector +import org.apache.griffin.measure.log.Loggable -trait DataConnector extends Serializable { + +trait DataConnector extends Loggable with Serializable { def available(): Boolean + def init(): Unit = {} + } diff --git a/measure/src/main/scala/org/apache/griffin/measure/connector/DataConnectorFactory.scala b/measure/src/main/scala/org/apache/griffin/measure/connector/DataConnectorFactory.scala index 589da9f56..82fea04b0 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/connector/DataConnectorFactory.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/connector/DataConnectorFactory.scala @@ -32,6 +32,25 @@ object DataConnectorFactory { val KafkaRegex = """^(?i)kafka$""".r + def getDataConnector(sqlContext: SQLContext, + ssc: StreamingContext, + dataConnectorParam: DataConnectorParam, + ruleExprs: RuleExprs, + globalFinalCacheMap: Map[String, Any] + ): Try[BatchDataConnector] = { + val conType = dataConnectorParam.conType + val version = dataConnectorParam.version + val config = dataConnectorParam.config + Try { + conType match { + case HiveRegex() => HiveBatchDataConnector(sqlContext, config, ruleExprs, globalFinalCacheMap) + case AvroRegex() => AvroBatchDataConnector(sqlContext, config, ruleExprs, globalFinalCacheMap) + case KafkaRegex() => KafkaDataConnector(sqlContext, ssc, dataConnectorParam, ruleExprs, globalFinalCacheMap) + case _ => throw new Exception("connector creation error!") + } + } + } + def getBatchDataConnector(sqlContext: SQLContext, dataConnectorParam: DataConnectorParam, ruleExprs: RuleExprs, @@ -44,15 +63,13 @@ object DataConnectorFactory { conType match { case HiveRegex() => HiveBatchDataConnector(sqlContext, config, ruleExprs, globalFinalCacheMap) case AvroRegex() => AvroBatchDataConnector(sqlContext, config, ruleExprs, globalFinalCacheMap) - case _ => throw new Exception("connector creation error!") + case _ => throw new Exception("batch connector creation error!") } } } def getStreamingDataConnector(ssc: StreamingContext, - dataConnectorParam: DataConnectorParam, - ruleExprs: RuleExprs, - globalFinalCacheMap: Map[String, Any] + dataConnectorParam: DataConnectorParam ): Try[StreamingDataConnector] = { val conType = dataConnectorParam.conType val version = dataConnectorParam.version @@ -62,7 +79,23 @@ object DataConnectorFactory { case KafkaRegex() => { genKafkaDataConnector(ssc, config) } - case _ => throw new Exception("connector creation error!") + case _ => throw new Exception("streaming connector creation error!") + } + } + } + + def getCacheDataConnector(sqlContext: SQLContext, + dataConnectorParam: DataConnectorParam + ): Try[CacheDataConnector] = { + val conType = dataConnectorParam.conType + val version = dataConnectorParam.version + val config = dataConnectorParam.config + Try { + conType match { + case KafkaRegex() => { + TempCacheDataConnector(sqlContext, config) + } + case _ => throw new Exception("cache connector creation error!") } } } @@ -74,19 +107,13 @@ object DataConnectorFactory { val valueType = config.getOrElse(ValueType, "java.lang.String").toString (getClassTag(keyType), getClassTag(valueType)) match { case (ClassTag(k: Class[String]), ClassTag(v: Class[String])) => { - new KafkaDataConnector(ssc, config) { + new KafkaStreamingDataConnector(ssc, config) { type K = String type KD = StringDecoder type V = String type VD = StringDecoder - - def stream(): Try[InputDStream[(K, V)]] = Try { - val topicSet = topics.split(",").toSet - KafkaUtils.createDirectStream[K, V, KD, VD]( - ssc, - kafkaConfig, - topicSet - ) + def createDStream(topicSet: Set[String]): InputDStream[(K, V)] = { + KafkaUtils.createDirectStream[K, V, KD, VD](ssc, kafkaConfig, topicSet) } } } diff --git a/measure/src/main/scala/org/apache/griffin/measure/connector/HiveBatchDataConnector.scala b/measure/src/main/scala/org/apache/griffin/measure/connector/HiveBatchDataConnector.scala index 26c6ff1fa..0318d9750 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/connector/HiveBatchDataConnector.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/connector/HiveBatchDataConnector.scala @@ -14,6 +14,7 @@ limitations under the License. */ package org.apache.griffin.measure.connector +import org.apache.griffin.measure.result._ import org.apache.griffin.measure.rule.{ExprValueUtil, RuleExprs} import org.apache.griffin.measure.rule.expr._ import org.apache.spark.rdd.RDD @@ -65,36 +66,61 @@ case class HiveBatchDataConnector(sqlContext: SQLContext, config: Map[String, An } } - def data(): Try[RDD[(Product, Map[String, Any])]] = { + def data(): Try[RDD[(Product, (Map[String, Any], Map[String, Any]))]] = { Try { sqlContext.sql(dataSql).flatMap { row => // generate cache data - val cacheExprValueMap: Map[String, Any] = ruleExprs.cacheExprs.foldLeft(constFinalExprValueMap) { (cachedMap, expr) => - ExprValueUtil.genExprValueMap(Some(row), expr, cachedMap) - } - val finalExprValueMap = ExprValueUtil.updateExprValueMap(ruleExprs.finalCacheExprs, cacheExprValueMap) - - // when clause filter data source - val whenResult = ruleExprs.whenClauseExprOpt match { - case Some(whenClause) => whenClause.calculate(finalExprValueMap) - case _ => None - } + val cacheExprValueMaps = ExprValueUtil.genExprValueMaps(Some(row), ruleExprs.cacheExprs, constFinalExprValueMap) + val finalExprValueMaps = ExprValueUtil.updateExprValueMaps(ruleExprs.finalCacheExprs, cacheExprValueMaps) + + // data info + val dataInfoMap: Map[String, Any] = DataInfo.cacheInfoList.map { info => + try { + (info.key -> row.getAs[info.T](info.key)) + } catch { + case e: Throwable => info.defWrap + } + }.toMap - // get groupby data - whenResult match { - case Some(false) => None - case _ => { - val groupbyData: Seq[AnyRef] = ruleExprs.groupbyExprs.flatMap { expr => - expr.calculate(finalExprValueMap) match { - case Some(v) => Some(v.asInstanceOf[AnyRef]) - case _ => None - } + finalExprValueMaps.flatMap { finalExprValueMap => + val groupbyData: Seq[AnyRef] = ruleExprs.groupbyExprs.flatMap { expr => + expr.calculate(finalExprValueMap) match { + case Some(v) => Some(v.asInstanceOf[AnyRef]) + case _ => None } - val key = toTuple(groupbyData) - - Some((key, finalExprValueMap)) } + val key = toTuple(groupbyData) + + Some((key, (finalExprValueMap, dataInfoMap))) } + + // generate cache data +// val cacheExprValueMap: Map[String, Any] = ruleExprs.cacheExprs.foldLeft(constFinalExprValueMap) { (cachedMap, expr) => +// ExprValueUtil.genExprValueMap(Some(row), expr, cachedMap) +// } +// val finalExprValueMap = ExprValueUtil.updateExprValueMap(ruleExprs.finalCacheExprs, cacheExprValueMap) +// +// // when clause filter data source +// val whenResult = ruleExprs.whenClauseExprOpt match { +// case Some(whenClause) => whenClause.calculate(finalExprValueMap) +// case _ => None +// } +// +// // get groupby data +// whenResult match { +// case Some(false) => None +// case _ => { +// val groupbyData: Seq[AnyRef] = ruleExprs.groupbyExprs.flatMap { expr => +// expr.calculate(finalExprValueMap) match { +// case Some(v) => Some(v.asInstanceOf[AnyRef]) +// case _ => None +// } +// } +// val key = toTuple(groupbyData) +// +// Some((key, finalExprValueMap)) +// } +// } } } } diff --git a/measure/src/main/scala/org/apache/griffin/measure/connector/KafkaDataConnector.scala b/measure/src/main/scala/org/apache/griffin/measure/connector/KafkaDataConnector.scala index 98644b3c7..603a05bd3 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/connector/KafkaDataConnector.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/connector/KafkaDataConnector.scala @@ -1,23 +1,216 @@ +/*- + * Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. + + */ package org.apache.griffin.measure.connector -import kafka.serializer.Decoder +import org.apache.griffin.measure.config.params.user.DataConnectorParam +import org.apache.griffin.measure.result._ +import org.apache.griffin.measure.rule.{DataTypeCalculationUtil, ExprValueUtil, RuleExprs} +import org.apache.griffin.measure.utils.TimeUtil +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.types.{DataType, StructField, StructType} import org.apache.spark.streaming.StreamingContext +import org.apache.spark.sql.{DataFrame, Row, SQLContext} + +import scala.util.{Failure, Success, Try} -abstract class KafkaDataConnector(ssc: StreamingContext, config: Map[String, Any] - ) extends StreamingDataConnector { - type KD <: Decoder[K] - type VD <: Decoder[V] +case class KafkaDataConnector(sqlContext: SQLContext, ssc: StreamingContext, dataConnectorParam: DataConnectorParam, + ruleExprs: RuleExprs, constFinalExprValueMap: Map[String, Any] + ) extends BatchDataConnector { - val KafkaConfig = "kafka.config" - val Topics = "topics" + val kafkaStreamingDataConnector = DataConnectorFactory.getStreamingDataConnector(ssc, dataConnectorParam) match { + case Success(cntr) => cntr + case Failure(ex) => throw ex + } + + val Cache = "cache" - val kafkaConfig = config.get(KafkaConfig) match { - case map: Map[String, Any] => map.mapValues(_.toString) - case _ => Map[String, String]() + val cacheDataConnectorParam = dataConnectorParam.config.get(Cache) match { + case Some(map: Map[String, Any]) => DataConnectorParam(map) + case _ => throw new Exception("invalid cache parameter!") } - val topics = config.getOrElse(Topics, "").toString + val cacheDataConnector = DataConnectorFactory.getCacheDataConnector(sqlContext, cacheDataConnectorParam) match { + case Success(cntr) => cntr + case Failure(ex) => throw ex + } + +// val DumpDatabase = "dump.database" +// val DumpTableName = "dump.table.name" +// val TempTableName = "temp.table.name" +// val TableNameRegex = """^[a-zA-Z\d][\w#@]{0,127}$""".r +// +// val dumpDatabase = dataConnectorParam.config.getOrElse(DumpDatabase, "").toString +// val (tempSave, useTempTable, dumpTableName, tempTableName) = { +// val (dump, dumpName) = useTable(DumpTableName) +// val (temp, tempName) = useTable(TempTableName) +// if (dump) { +// (false, dumpName, tempName) +// } else if (temp) { +// (true, dumpName, tempName) +// } else throw new Exception("invalid dump table name and temporary table name!") +// } + +// private def useTable(key: String): (Boolean, String) = { +// dataConnectorParam.config.get(key) match { +// case Some(name: String) => { +// name match { +// case TableNameRegex() => (true, name) +// case _ => (false, name) +// } +// } +// case _ => (false, "") +// } +// } def available(): Boolean = { - true + kafkaStreamingDataConnector.available && cacheDataConnector.available + } + + override def init(): Unit = { + val ds = kafkaStreamingDataConnector.stream match { + case Success(dstream) => dstream + case Failure(ex) => throw ex + } + ds.foreachRDD((rdd, time) => { + val ms = time.milliseconds +// val min = TimeUtil.timeToUnit(ms, "min") +// val hour = TimeUtil.timeToUnit(ms, "hour") +// val partitions = List[(String, Any)](("hr", hour), ("min", min)) + +// val partitionPath = genPartitionHdfsPath(partitions) +// val path = s"${targetDumpDir}/${partitionPath}/${ms}" + + val dataInfoMap = DataInfo.cacheInfoList.map(_.defWrap).toMap + TimeStampInfo.wrap(ms) + + // parse each message + val valueMapRdd: RDD[Map[String, Any]] = rdd.flatMap { kv => + val msg = kv._2 + + val cacheExprValueMaps = ExprValueUtil.genExprValueMaps(Some(msg), ruleExprs.cacheExprs, constFinalExprValueMap) + val finalExprValueMaps = ExprValueUtil.updateExprValueMaps(ruleExprs.finalCacheExprs, cacheExprValueMaps) + +// val sf = StructField("name", DataType.fromJson("string")) +// val schema: StructType = new StructType() + + finalExprValueMaps.map { vm => + vm ++ dataInfoMap + } + } + + // generate DataFrame + val df = genDataFrame(valueMapRdd) + + // save data frame + cacheDataConnector.saveData(df, ms) + }) + } + + // generate DataFrame + // maybe we can directly use def createDataFrame[A <: Product : TypeTag](rdd: RDD[A]): DataFrame + // to avoid generate data type by myself, just translate each value into Product + private def genDataFrame(rdd: RDD[Map[String, Any]]): DataFrame = { + val fields = rdd.aggregate(Map[String, DataType]())( + DataTypeCalculationUtil.sequenceDataTypeMap, DataTypeCalculationUtil.combineDataTypeMap + ).toList.map(f => StructField(f._1, f._2)) + val schema = StructType(fields) + val datas: RDD[Row] = rdd.map { d => + val values = fields.map { field => + val StructField(k, dt, _, _) = field + d.get(k) match { + case Some(v) => v + case _ => null + } + } + Row(values: _*) + } + val df = sqlContext.createDataFrame(datas, schema) + df + } + + def metaData(): Try[Iterable[(String, String)]] = Try { + Map.empty[String, String] + } + + def data(): Try[RDD[(Product, (Map[String, Any], Map[String, Any]))]] = Try { + cacheDataConnector.readData.flatMap { row => + // generate cache data + val cacheExprValueMaps = ExprValueUtil.genExprValueMaps(Some(row), ruleExprs.cacheExprs, constFinalExprValueMap) + val finalExprValueMaps = ExprValueUtil.updateExprValueMaps(ruleExprs.finalCacheExprs, cacheExprValueMaps) + + // data info + val dataInfoMap: Map[String, Any] = DataInfo.cacheInfoList.map { info => + try { + (info.key -> row.getAs[info.T](info.key)) + } catch { + case e: Throwable => info.defWrap + } + }.toMap + + finalExprValueMaps.flatMap { finalExprValueMap => + val groupbyData: Seq[AnyRef] = ruleExprs.groupbyExprs.flatMap { expr => + expr.calculate(finalExprValueMap) match { + case Some(v) => Some(v.asInstanceOf[AnyRef]) + case _ => None + } + } + val key = toTuple(groupbyData) + + Some((key, (finalExprValueMap, dataInfoMap))) + } + } } -} \ No newline at end of file + +// private def dbPrefix(): Boolean = { +// if (useTempTable) false else dumpDatabase.nonEmpty && !dumpDatabase.equals("default") +// } +// +// private def fullDumpTableName: String = if (dbPrefix) dumpTableName else s"${dumpDatabase}.${dumpTableName}" +// +// private def dumpTableExists(): Boolean = { +// (!dumpTableName.isEmpty) && { +// Try { +// if (dbPrefix) { +// sqlContext.tables(dumpDatabase).filter(dumpTableExistsSql).collect.size +// } else { +// sqlContext.tables().filter(dumpTableExistsSql).collect.size +// } +// } match { +// case Success(s) => s > 0 +// case _ => false +// } +// } +// } +// +// private def dumpTableExistsSql(): String = { +// s"tableName LIKE '${dumpTableName}'" +// } +// +// private def createDumpTableSql(df: DataFrame): Unit = { +//// df.schema. +//// s"CREATE TABLE IF NOT EXISTS ${fullDumpTableName} " +// } +// +// private def saveTempDataFrame(df: DataFrame): Unit = { +// df.registerTempTable() +// s"CREATE TABLE IF NOT EXISTS ${fullTempTableName} " +// } + + private def toTuple[A <: AnyRef](as: Seq[A]): Product = { + if (as.size > 0) { + val tupleClass = Class.forName("scala.Tuple" + as.size) + tupleClass.getConstructors.apply(0).newInstance(as: _*).asInstanceOf[Product] + } else None + } + +} diff --git a/measure/src/main/scala/org/apache/griffin/measure/connector/KafkaStreamingDataConnector.scala b/measure/src/main/scala/org/apache/griffin/measure/connector/KafkaStreamingDataConnector.scala new file mode 100644 index 000000000..1db8dc513 --- /dev/null +++ b/measure/src/main/scala/org/apache/griffin/measure/connector/KafkaStreamingDataConnector.scala @@ -0,0 +1,49 @@ +/*- + * Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. + + */ +package org.apache.griffin.measure.connector + +import kafka.serializer.Decoder +import org.apache.spark.streaming.StreamingContext +import org.apache.spark.streaming.dstream.InputDStream +import org.apache.spark.streaming.kafka.KafkaUtils + +import scala.reflect.ClassTag +import scala.util.Try + +abstract class KafkaStreamingDataConnector(ssc: StreamingContext, config: Map[String, Any] + ) extends StreamingDataConnector { + type KD <: Decoder[K] + type VD <: Decoder[V] + + val KafkaConfig = "kafka.config" + val Topics = "topics" + + val kafkaConfig = config.get(KafkaConfig) match { + case map: Map[String, Any] => map.mapValues(_.toString) + case _ => Map[String, String]() + } + val topics = config.getOrElse(Topics, "").toString + + def available(): Boolean = { + true + } + + def stream(): Try[InputDStream[(K, V)]] = Try { + val topicSet = topics.split(",").toSet + createDStream(topicSet) + } + + protected def createDStream(topicSet: Set[String]): InputDStream[(K, V)] +} \ No newline at end of file diff --git a/measure/src/main/scala/org/apache/griffin/measure/connector/TempCacheDataConnector.scala b/measure/src/main/scala/org/apache/griffin/measure/connector/TempCacheDataConnector.scala new file mode 100644 index 000000000..1dec51b27 --- /dev/null +++ b/measure/src/main/scala/org/apache/griffin/measure/connector/TempCacheDataConnector.scala @@ -0,0 +1,63 @@ +package org.apache.griffin.measure.connector + +import org.apache.spark.sql.{DataFrame, SQLContext} + +import scala.util.{Try, Success} + +case class TempCacheDataConnector(sqlContext: SQLContext, config: Map[String, Any] + ) extends CacheDataConnector { + + val TableName = "table.name" + val tableName = config.getOrElse(TableName, "").toString + val tmpTableName = s"${tableName}_tmp" + + val timeStampColumn = "_tmst_" + + var tableCreated = tableExists + + val InfoPath = "info.path" + val cacheTimeKey: String = config.getOrElse(InfoPath, "").toString + + def available(): Boolean = { + tableName.nonEmpty + } + + def saveData(df: DataFrame, ms: Long): Unit = { + if (!tableCreated) { + df.registerTempTable(tableName) + sqlContext.cacheTable(tableName) + tableCreated = true + } else { + df.registerTempTable(tmpTableName) + sqlContext.sql(s"INSERT INTO TABLE ${tableName} SELECT * FROM ${tmpTableName}") + sqlContext.dropTempTable(tmpTableName) + } + + // submit ms + submitCacheTime(ms) + } + + def readData(): DataFrame = { + val timeRange = readTimeRange + val readSql = readDataSql(timeRange) + sqlContext.sql(readSql) + } + + private def tableExists(): Boolean = { + Try { + sqlContext.tables().filter(tableExistsSql).collect.size + } match { + case Success(s) => s > 0 + case _ => false + } + } + + private def tableExistsSql(): String = { + s"tableName LIKE '${tableName}'" + } + + private def readDataSql(timeRange: (Long, Long)): String = { + s"SELECT * FROM ${tableName} WHERE `${timeStampColumn}` BETWEEN ${timeRange._1} AND ${timeRange._2}" + } + +} diff --git a/measure/src/main/scala/org/apache/griffin/measure/persist/PersistFactory.scala b/measure/src/main/scala/org/apache/griffin/measure/persist/PersistFactory.scala index cae6730d6..43bb2a83e 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/persist/PersistFactory.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/persist/PersistFactory.scala @@ -31,11 +31,11 @@ case class PersistFactory(persistParams: Iterable[PersistParam], metricName: Str // get the persists configured private def getPersist(timeStamp: Long, persistParam: PersistParam): Option[Persist] = { - val persistConfig = persistParam.config + val config = persistParam.config val persistTry = persistParam.persistType match { - case HDFS_REGEX() => Try(HdfsPersist(persistConfig, metricName, timeStamp)) - case HTTP_REGEX() => Try(HttpPersist(persistConfig, metricName, timeStamp)) - case LOG_REGEX() => Try(LoggerPersist(persistConfig, metricName, timeStamp)) + case HDFS_REGEX() => Try(HdfsPersist(config, metricName, timeStamp)) + case HTTP_REGEX() => Try(HttpPersist(config, metricName, timeStamp)) + case LOG_REGEX() => Try(LoggerPersist(config, metricName, timeStamp)) case _ => throw new Exception("not supported persist type") } persistTry match { diff --git a/measure/src/main/scala/org/apache/griffin/measure/result/ResultInfo.scala b/measure/src/main/scala/org/apache/griffin/measure/result/DataInfo.scala similarity index 54% rename from measure/src/main/scala/org/apache/griffin/measure/result/ResultInfo.scala rename to measure/src/main/scala/org/apache/griffin/measure/result/DataInfo.scala index 2020caf64..e2cf5e17c 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/result/ResultInfo.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/result/DataInfo.scala @@ -15,39 +15,32 @@ limitations under the License. package org.apache.griffin.measure.result -sealed trait ResultInfo { +sealed trait DataInfo { type T val key: String - val tp: String def wrap(value: T) = (key -> value) + def defWrap() = wrap(dfv) + val dfv: T } -final case object TimeGroupInfo extends ResultInfo { +final case object TimeStampInfo extends DataInfo { type T = Long - val key = "__time__" - val tp = "bigint" + val key = "_tmst_" + val dfv = 0L } -final case object NextFireTimeInfo extends ResultInfo { - type T = Long - val key = "__next_fire_time__" - val tp = "bigint" -} - -final case object MismatchInfo extends ResultInfo { +final case object MismatchInfo extends DataInfo { type T = String - val key = "__mismatch__" - val tp = "string" + val key = "_mismatch_" + val dfv = "" } -final case object TargetInfo extends ResultInfo { - type T = Map[String, Any] - val key = "__target__" - val tp = "map" -} - -final case object ErrorInfo extends ResultInfo { +final case object ErrorInfo extends DataInfo { type T = String - val key = "__error__" - val tp = "string" + val key = "_error_" + val dfv = "" } + +object DataInfo { + val cacheInfoList = List(TimeStampInfo, MismatchInfo, ErrorInfo) +} \ No newline at end of file diff --git a/measure/src/main/scala/org/apache/griffin/measure/rule/DataTypeCalculationUtil.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/DataTypeCalculationUtil.scala new file mode 100644 index 000000000..1e865699e --- /dev/null +++ b/measure/src/main/scala/org/apache/griffin/measure/rule/DataTypeCalculationUtil.scala @@ -0,0 +1,114 @@ +package org.apache.griffin.measure.rule + +import org.apache.spark.sql.types._ + +object DataTypeCalculationUtil { + + implicit def dataType2CalculationType(tp: DataType): CalculationType = CalculationType(tp) + + case class CalculationType(tp: DataType) extends Serializable { + def binaryOpr (other: DataType): DataType = { + (tp, other) match { + case (NullType, _) | (_, NullType) => NullType + case (t, _) => t + } + } + def unaryOpr (): DataType = { + tp + } + } + + case class DataTypeException() extends Exception {} + + def getDataType(value: Any): DataType = { + value match { + case v: String => StringType + case v: Boolean => BooleanType + case v: Long => LongType + case v: Int => LongType + case v: Short => LongType + case v: Byte => LongType + case v: Double => DoubleType + case v: Float => DoubleType + case v: Map[_, _] => MapType(getSameDataType(v.keys), getSameDataType(v.values)) + case v: Iterable[_] => ArrayType(getSameDataType(v)) + case _ => NullType + } + } + + private def getSameDataType(values: Iterable[Any]): DataType = { + values.foldLeft(NullType: DataType)((a, c) => genericTypeOf(a, getDataType(c))) + } + + private def genericTypeOf(dt1: DataType, dt2: DataType): DataType = { + if (dt1 == dt2) dt1 else { + dt1 match { + case NullType => dt2 + case StringType => StringType + case DoubleType => { + dt2 match { + case StringType => StringType + case DoubleType | LongType => DoubleType + case _ => throw DataTypeException() + } + } + case LongType => { + dt2 match { + case StringType => StringType + case DoubleType => DoubleType + case LongType => LongType + case _ => throw DataTypeException() + } + } + case BooleanType => { + dt2 match { + case StringType => StringType + case BooleanType => BooleanType + case _ => throw DataTypeException() + } + } + case MapType(kdt1, vdt1, _) => { + dt2 match { + case MapType(kdt2, vdt2, _) => MapType(genericTypeOf(kdt1, kdt2), genericTypeOf(vdt1, vdt2)) + case _ => throw DataTypeException() + } + } + case ArrayType(vdt1, _) => { + dt2 match { + case ArrayType(vdt2, _) => ArrayType(genericTypeOf(vdt1, vdt2)) + case _ => throw DataTypeException() + } + } + case _ => throw DataTypeException() + } + } + } + + def sequenceDataTypeMap(aggr: Map[String, DataType], value: Map[String, Any]): Map[String, DataType] = { + val dataTypes = value.foldLeft(Map[String, DataType]()) { (map, pair) => + val (k, v) = pair + try { + map + (k -> getDataType(v)) + } catch { + case e: DataTypeException => map + } + } + combineDataTypeMap(aggr, dataTypes) + } + + def combineDataTypeMap(aggr1: Map[String, DataType], aggr2: Map[String, DataType]): Map[String, DataType] = { + aggr2.foldLeft(aggr1) { (a, c) => + a.get(c._1) match { + case Some(t) => { + try { + a + (c._1 -> genericTypeOf(t, c._2)) + } catch { + case e: DataTypeException => a + } + } + case _ => a + c + } + } + } + +} diff --git a/measure/src/main/scala/org/apache/griffin/measure/rule/ExprValueUtil.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/ExprValueUtil.scala index 8d97ee4d7..f5739f9e0 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rule/ExprValueUtil.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rule/ExprValueUtil.scala @@ -15,75 +15,236 @@ limitations under the License. package org.apache.griffin.measure.rule import org.apache.griffin.measure.rule.expr._ +import org.apache.griffin.measure.rule.func._ import org.apache.spark.sql.Row import scala.util.{Success, Try} object ExprValueUtil { +// private def calcExprValue(originDatas: Seq[Option[Any]], expr: Expr, existExprValueMap: Map[String, Any]): Seq[Option[Any]] = { +// originDatas.flatMap { originData => +// calcExprValue(originData, expr, existExprValueMap) +// } +// } + // from origin data such as a Row of DataFrame, with existed expr value map, calculate related expression, get the expression value // for now, one expr only get one value, not supporting one expr get multiple values // params: // - originData: the origin data such as a Row of DataFrame // - expr: the expression to be calculated // - existExprValueMap: existed expression value map, which might be used to get some existed expression value during calculation - // output: the calculated expression value - private def calcExprValue(originData: Option[Any], expr: Expr, existExprValueMap: Map[String, Any]): Option[Any] = { + // output: the calculated expression values +// private def calcExprValue(originData: Option[Any], expr: Expr, existExprValueMap: Map[String, Any]): Seq[Option[Any]] = { +// Try { +// expr match { +// case selection: SelectionExpr => { +// selection.selectors.foldLeft(Seq(originData)) { (datas, selector) => +// calcExprValue(datas, selector, existExprValueMap) +// } +// } +// case selector: IndexFieldRangeSelectExpr => { +// originData match { +// case Some(row: Row) => { +// if (selector.fields.size == 1) { +// selector.fields.head match { +// case i: IndexDesc => Seq(Some(row.getAs[Any](i.index))) +// case f: FieldDesc => Seq(Some(row.getAs[Any](f.field))) +// case _ => Nil +// } +// } else Nil +// } +// case Some(d: Map[String, Any]) => { +// selector.fields.foldLeft(Seq[Option[Any]]()) { (results, field) => +// results ++ (field match { +// case f: FieldDesc => opt2Seq(d.get(f.field)) +// case a: AllFieldsDesc => d.values.map(Some(_)).toSeq +// case _ => Nil +// }) +// } +// } +// case Some(d: Seq[Any]) => { +// selector.fields.foldLeft(Seq[Option[Any]]()) { (results, field) => +// results ++ (field match { +// case i: IndexDesc => opt2Seq(try { Some(d(i.index)) } catch { case _ => None }) +// case a: AllFieldsDesc => d.map(Some(_)) +// case r: FieldRangeDesc => Nil // not done +// case _ => Nil +// }) +// } +// } +// case _ => Nil +// } +// } +// case selector: FunctionOperationExpr => { +// val args: Array[Option[Any]] = selector.args.map { arg => +// arg.calculate(existExprValueMap) +// }.toArray +// originData match { +// case Some(d: String) => { +// FunctionUtil.invoke(selector.func, Some(d) +: args) +// } +// case _ => Nil +// } +// } +// case _ => Seq(expr.calculate(existExprValueMap)) +// } +// } match { +// case Success(v) => v +// case _ => Nil +// } +// } + + private def append(path: List[String], step: String): List[String] = { + path :+ step + } + + private def calcExprValues(pathDatas: List[(List[String], Option[Any])], expr: Expr, existExprValueMap: Map[String, Any]): List[(List[String], Option[Any])] = { Try { expr match { case selection: SelectionExpr => { - selection.selectors.foldLeft(originData) { (dt, selector) => - calcExprValue(dt, selector, existExprValueMap) + selection.selectors.foldLeft(pathDatas) { (pds, selector) => + calcExprValues(pds, selector, existExprValueMap) } } case selector: IndexFieldRangeSelectExpr => { - originData match { - case Some(row: Row) => { - if (selector.fields.size == 1) { - selector.fields.head match { - case i: IndexDesc => Some(row.getAs[Any](i.index)) - case f: FieldDesc => Some(row.getAs[Any](f.field)) - case _ => None + pathDatas.flatMap { pathData => + val (path, data) = pathData + data match { + case Some(row: Row) => { + selector.fields.flatMap { field => + field match { + case i: IndexDesc => Some((append(path, i.desc), Some(row.getAs[Any](i.index)))) + case f: FieldDesc => Some((append(path, f.desc), Some(row.getAs[Any](f.field)))) + case _ => None + } + } + } + case Some(d: Map[String, Any]) => { + selector.fields.flatMap { field => + field match { + case f: FieldDesc => Some((append(path, f.desc), d.get(f.field))) + case a: AllFieldsDesc => { + d.map { kv => + val (k, v) = kv + (append(path, s"${a.desc}_${k}"), Some(v)) + } + } + case _ => None + } } - } else None + } + case Some(d: Seq[Any]) => { + selector.fields.flatMap { field => + field match { + case i: IndexDesc => { + if (i.index >= 0 && i.index < d.size) { + Some((append(path, i.desc), Some(d(i.index)))) + } else None + } + case a: AllFieldsDesc => { + val dt = d.zipWithIndex + dt.map { kv => + val (v, i) = kv + (append(path, s"${a.desc}_${i}"), Some(v)) + } + } + case r: FieldRangeDesc => { + (r.startField, r.endField) match { + case (si: IndexDesc, ei: IndexDesc) => { + if (si.index >= 0 && ei.index < d.size && si.index <= ei.index) { + val dt = d.zipWithIndex + dt.filter(kv => (kv._2 >= si.index && kv._2 <= ei.index)).map { kv => + val (v, i) = kv + (append(path, s"${r.desc}_${i}"), Some(v)) + } + } else None + } + } + } + case _ => None + } + } + } } - case _ => None } } - case _ => expr.calculate(existExprValueMap) + case selector: FunctionOperationExpr => { + val args: Array[Option[Any]] = selector.args.map { arg => + arg.calculate(existExprValueMap) + }.toArray + pathDatas.flatMap { pathData => + val (path, data) = pathData + data match { + case Some(d: String) => { + val res = FunctionUtil.invoke(selector.func, Some(d) +: args) + val residx = res.zipWithIndex + residx.map { vi => + val (v, i) = vi + val step = if (i == 0) s"${selector.desc}" else s"${selector.desc}_${i}" + (append(path, step), v) + } + } + case _ => None + } + } + } + case _ => { + (expr.desc :: Nil, expr.calculate(existExprValueMap)) :: Nil + } } } match { case Success(v) => v - case _ => None + case _ => Nil } } - // try to calculate expr from data and initExprValueMap, generate a new expression value map - // depends on origin data and existed expr value map - def genExprValueMap(data: Option[Any], expr: Expr, initExprValueMap: Map[String, Any]): Map[String, Any] = { - val valueOpt = calcExprValue(data, expr, initExprValueMap) - if (valueOpt.nonEmpty) { - initExprValueMap + (expr._id -> valueOpt.get) - } else initExprValueMap + private def calcExprsValues(data: Option[Any], exprs: Iterable[Expr], existExprValueMap: Map[String, Any]): List[Map[String, Any]] = { + val schemaValues: Map[String, List[(List[String], Any)]] = exprs.map { expr => + (expr._id, calcExprValues((Nil, data) :: Nil, expr, existExprValueMap).flatMap { pair => + pair._2 match { + case Some(v) => Some((pair._1, v)) + case _ => None + } + }) + }.toMap + SchemaValueCombineUtil.cartesian(schemaValues) } + // try to calculate expr from data and initExprValueMap, generate new expression value maps + // depends on origin data and existed expr value map +// def genExprValueMap(data: Option[Any], expr: Expr, initExprValueMap: Map[String, Any]): Seq[Map[String, Any]] = { +// val valueOpts = calcExprValues(data, expr, initExprValueMap) +// valueOpts.map { valueOpt => +// if (valueOpt.nonEmpty) { +// initExprValueMap + (expr._id -> valueOpt.get) +// } else initExprValueMap +// } +// } + // try to calculate some exprs from data and initExprValueMap, generate a new expression value map // depends on origin data and existed expr value map - def genExprValueMap(data: Option[Any], exprs: Iterable[Expr], initExprValueMap: Map[String, Any]): Map[String, Any] = { - exprs.foldLeft(initExprValueMap) { (evMap, expr) => - ExprValueUtil.genExprValueMap(None, expr, evMap) + def genExprValueMaps(data: Option[Any], exprs: Iterable[Expr], initExprValueMap: Map[String, Any]): List[Map[String, Any]] = { + val valueMaps = calcExprsValues(data, exprs, initExprValueMap) + + valueMaps.map { valueMap => + initExprValueMap ++ valueMap } } // with exprValueMap, calculate expressions, update the expression value map // only depends on existed expr value map, only calculation, not need origin data - def updateExprValueMap(exprs: Iterable[Expr], exprValueMap: Map[String, Any]): Map[String, Any] = { - exprs.foldLeft(Map[String, Any]()) { (evMap, expr) => - val valueOpt = expr.calculate(exprValueMap) - if (valueOpt.nonEmpty) { - evMap + (expr._id -> valueOpt.get) - } else evMap + def updateExprValueMaps(exprs: Iterable[Expr], exprValueMaps: List[Map[String, Any]]): List[Map[String, Any]] = { + exprValueMaps.flatMap { exprValueMap => + genExprValueMaps(None, exprs, exprValueMap) } } +// private def opt2Seq(opt: Option[Any]): Seq[Option[Any]] = { +// opt match { +// case Some(v) => Seq(opt) +// case _ => Nil +// } +// } + } diff --git a/measure/src/main/scala/org/apache/griffin/measure/rule/SchemaValueCombineUtil.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/SchemaValueCombineUtil.scala new file mode 100644 index 000000000..00a794c5a --- /dev/null +++ b/measure/src/main/scala/org/apache/griffin/measure/rule/SchemaValueCombineUtil.scala @@ -0,0 +1,196 @@ +package org.apache.griffin.measure.rule + +object SchemaValueCombineUtil { + + // Map[String, List[(List[String], T)]]: Map[key, List[(path, value)]] + def cartesian[T](valuesMap: Map[String, List[(List[String], T)]]): List[Map[String, T]] = { + val fieldsList: List[(String, List[(List[String], T)])] = valuesMap.toList + + // wrong algorithm: assume the lists have same size +// val minSize = fieldsList.map(_._2.size).min +// val idxes = (0 to (minSize - 1)).toList +// idxes.map { idx => +// fieldsList.foldLeft(Map[String, T]()) { (map, pair) => +// val (key, value) = pair +// map + (key -> value(idx)._2) +// } +// } + + // following is correct algorithm + + // List[key, List[(path, value)]] to List[(path, (key, value))] + val valueList: List[(List[String], (String, T))] = fieldsList.flatMap { fields => + val (key, list) = fields + list.map { pv => + val (path, value) = pv + (path, (key, value)) + } + } + + // 1. generate tree from value list, and return root node + val root = TreeUtil.genRootTree(valueList) + + // 2. deep first visit tree from root, merge datas into value map list + val valueMapList: List[Map[String, _]] = TreeUtil.mergeDatasIntoMap(root, Nil) + + // 3. simple change + valueMapList.map { mp => + mp.map { kv => + val (k, v) = kv + (k, v.asInstanceOf[T]) + } + } + + } + +// def cartesianTest[T](valuesMap: Map[String, List[(List[String], T)]]): Unit = { +// val fieldsList: List[(String, List[(List[String], T)])] = valuesMap.toList +// +// // List[key, List[(path, value)]] to List[(path, (key, value))] +// val valueList: List[(List[String], (String, T))] = fieldsList.flatMap { fields => +// val (key, list) = fields +// list.map { pv => +// val (path, value) = pv +// (path, (key, value)) +// } +// } +// +// // 1. generate tree from value list, and return root node +// val root = TreeUtil.genRootTree(valueList) +// +// // 2. deep first visit tree from root, merge datas into value map list +// val valueMapList: List[Map[String, _]] = TreeUtil.mergeDatasIntoMap(root, Nil) +// +// valueMapList.foreach(println) +// +// } + + + case class TreeNode(key: String, var datas: List[(String, _)]) { + var children = List[TreeNode]() + def addChild(node: TreeNode): Unit = children = children :+ node + def mergeSelf(node: TreeNode): Unit = datas = datas ::: node.datas + } + + object TreeUtil { + private def genTree(path: List[String], datas: List[(String, _)]): Option[TreeNode] = { + path match { + case Nil => None +// case head :: Nil => Some(TreeNode(datas)) + case head :: tail => { + genTree(tail, datas) match { + case Some(child) => { + val curNode = TreeNode(head, Nil) + curNode.addChild(child) + Some(curNode) + } + case _ => Some(TreeNode(head, datas)) + } + } + } + } + + private def mergeTrees(trees: List[TreeNode], newTreeOpt: Option[TreeNode]): List[TreeNode] = { + newTreeOpt match { + case Some(newTree) => { + trees.find(tree => tree.key == newTree.key) match { + case Some(tree) => { + // children merge + for (child <- newTree.children) { + tree.children = mergeTrees(tree.children, Some(child)) + } + // self data merge + tree.mergeSelf(newTree) + trees + } + case _ => trees :+ newTree + } + } + case _ => trees + } + } + + private def root(): TreeNode = TreeNode("", Nil) + + def genRootTree(values: List[(List[String], (String, _))]): TreeNode = { + val rootNode = root() + val nodeOpts = values.map(value => genTree(value._1, value._2 :: Nil)) + rootNode.children = nodeOpts.foldLeft(List[TreeNode]()) { (trees, treeOpt) => + mergeTrees(trees, treeOpt) + } + rootNode + } + + private def add(mapList1: List[Map[String, _]], mapList2: List[Map[String, _]]): List[Map[String, _]] = { + mapList1 ::: mapList2 + } + private def multiply(mapList1: List[Map[String, _]], mapList2: List[Map[String, _]]): List[Map[String, _]] = { + mapList1.flatMap { map1 => + mapList2.map { map2 => + map1 ++ map2 + } + } + } + + private def keysList(mapList: List[Map[String, _]]): List[String] = { + val keySet = mapList match { + case Nil => Set[String]() + case head :: _ => head.keySet + } + keySet.toList + } + + def mergeDatasIntoMap(root: TreeNode, mapDatas: List[Map[String, _]]): List[Map[String, _]] = { + val childrenKeysMapDatas = root.children.foldLeft(Map[List[String], List[Map[String, _]]]()) { (keysMap, child) => + val childMdts = mergeDatasIntoMap(child, List[Map[String, _]]()) + val keys = keysList(childMdts) + val afterList = keysMap.get(keys) match { + case Some(list) => add(list, childMdts) + case _ => childMdts + } + keysMap + (keys -> afterList) + } + val childrenMergeMaps = childrenKeysMapDatas.values.foldLeft(List[Map[String, _]]()) { (originList, list) => + originList match { + case Nil => list + case _ => multiply(originList, list) + } + } + mergeNodeChildrenDatasIntoMap(root, childrenMergeMaps) + } + + private def mergeNodeChildrenDatasIntoMap(node: TreeNode, mapDatas: List[Map[String, _]]): List[Map[String, _]] = { + val datas: List[(String, (String, Any))] = node.children.flatMap { child => + child.datas.map(dt => (dt._1, (child.key, dt._2))) + } + val childrenDataKeys: Set[String] = datas.map(_._1).toSet + val childrenDataLists: Map[String, List[(String, _)]] = datas.foldLeft(childrenDataKeys.map(k => (k, List[(String, _)]())).toMap) { (maps, data) => + maps.get(data._1) match { + case Some(list) => maps + (data._1 -> (list :+ data._2)) + case _ => maps + } + } + + // multiply different key datas + childrenDataLists.foldLeft(mapDatas) { (mdts, klPair) => + val (key, list) = klPair + mdts match { + case Nil => list.map(pr => Map[String, Any]((key -> pr._2))) + case _ => { + list.flatMap { kvPair => + val (path, value) = kvPair + mdts.map { mp => + mp + (key -> value) + } + } + } + } + } + + } + } + + + + +} diff --git a/measure/src/main/scala/org/apache/griffin/measure/rule/expr/Expr.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/expr/Expr.scala index 74ec4573f..805f2b9ab 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rule/expr/Expr.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rule/expr/Expr.scala @@ -14,12 +14,16 @@ limitations under the License. */ package org.apache.griffin.measure.rule.expr +import org.apache.spark.sql.types.DataType + trait Expr extends Serializable with Describable with Cacheable with Calculatable { protected val _defaultId: String = ExprIdCounter.emptyId val _id = ExprIdCounter.genId(_defaultId) + def dataType: DataType + protected def getSubCacheExprs(ds: String): Iterable[Expr] = Nil final def getCacheExprs(ds: String): Iterable[Expr] = { if (cacheable(ds)) getSubCacheExprs(ds).toList :+ this else getSubCacheExprs(ds) diff --git a/measure/src/main/scala/org/apache/griffin/measure/rule/expr/LiteralExpr.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/expr/LiteralExpr.scala index 49e9d8284..9dfd295de 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rule/expr/LiteralExpr.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rule/expr/LiteralExpr.scala @@ -15,6 +15,7 @@ limitations under the License. package org.apache.griffin.measure.rule.expr import org.apache.griffin.measure.utils.TimeUtil +import org.apache.spark.sql.types._ import scala.util.{Failure, Success, Try} @@ -27,6 +28,7 @@ trait LiteralExpr extends Expr { case class LiteralStringExpr(expr: String) extends LiteralExpr { val value: Option[String] = Some(expr) val desc: String = s"'${value.getOrElse("")}'" + def dataType: DataType = StringType } case class LiteralNumberExpr(expr: String) extends LiteralExpr { @@ -44,12 +46,14 @@ case class LiteralNumberExpr(expr: String) extends LiteralExpr { } } val desc: String = value.getOrElse("").toString + def dataType: DataType = DoubleType } case class LiteralTimeExpr(expr: String) extends LiteralExpr { final val TimeRegex = """(\d+)(d|h|m|s|ms)""".r val value: Option[Long] = TimeUtil.milliseconds(expr) val desc: String = expr + def dataType: DataType = LongType } case class LiteralBooleanExpr(expr: String) extends LiteralExpr { @@ -61,14 +65,17 @@ case class LiteralBooleanExpr(expr: String) extends LiteralExpr { case _ => throw new Exception(s"${expr} is invalid boolean") } val desc: String = value.getOrElse("").toString + def dataType: DataType = BooleanType } case class LiteralNullExpr(expr: String) extends LiteralExpr { val value: Option[Any] = Some(null) val desc: String = "null" + def dataType: DataType = NullType } case class LiteralNoneExpr(expr: String) extends LiteralExpr { val value: Option[Any] = None val desc: String = "none" + def dataType: DataType = NullType } \ No newline at end of file diff --git a/measure/src/main/scala/org/apache/griffin/measure/rule/expr/LogicalExpr.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/expr/LogicalExpr.scala index 95cf19236..ca99fb2b2 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rule/expr/LogicalExpr.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rule/expr/LogicalExpr.scala @@ -15,9 +15,11 @@ limitations under the License. package org.apache.griffin.measure.rule.expr import org.apache.griffin.measure.rule.CalculationUtil._ +import org.apache.spark.sql.types.{BooleanType, DataType} trait LogicalExpr extends Expr with AnalyzableExpr { override def cacheUnit: Boolean = true + def dataType: DataType = BooleanType } case class LogicalSimpleExpr(expr: MathExpr) extends LogicalExpr { diff --git a/measure/src/main/scala/org/apache/griffin/measure/rule/expr/MathExpr.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/expr/MathExpr.scala index ec8645ae0..5cc225b24 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rule/expr/MathExpr.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rule/expr/MathExpr.scala @@ -15,6 +15,8 @@ limitations under the License. package org.apache.griffin.measure.rule.expr import org.apache.griffin.measure.rule.CalculationUtil._ +import org.apache.griffin.measure.rule.DataTypeCalculationUtil._ +import org.apache.spark.sql.types.DataType trait MathExpr extends Expr { @@ -22,6 +24,7 @@ trait MathExpr extends Expr { case class MathFactorExpr(self: Expr) extends MathExpr { def calculateOnly(values: Map[String, Any]): Option[Any] = self.calculate(values) + def dataType: DataType = self.dataType val desc: String = self.desc val dataSources: Set[String] = self.dataSources override def getSubCacheExprs(ds: String): Iterable[Expr] = { @@ -47,6 +50,11 @@ case class UnaryMathExpr(oprList: Iterable[String], factor: Expr) extends MathEx } } } + def dataType: DataType = { + oprList.foldRight(factor.dataType unaryOpr) { (opr, tp) => + tp unaryOpr + } + } val desc: String = oprList.foldRight(factor.desc) { (prev, ex) => s"${prev}${ex}" } val dataSources: Set[String] = factor.dataSources override def cacheUnit: Boolean = true @@ -78,6 +86,13 @@ case class BinaryMathExpr(first: MathExpr, others: Iterable[(String, MathExpr)]) } } } + def dataType: DataType = { + others.foldLeft(first.dataType) { (tp, pair) => + val (opr, next) = pair + val ntp = next.dataType + tp binaryOpr ntp + } + } val desc: String = others.foldLeft(first.desc) { (ex, next) => s"${ex} ${next._1} ${next._2.desc}" } val dataSources: Set[String] = first.dataSources ++ others.flatMap(_._2.dataSources).toSet override def cacheUnit: Boolean = true diff --git a/measure/src/main/scala/org/apache/griffin/measure/rule/func/DefaultFunctionDefine.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/func/DefaultFunctionDefine.scala new file mode 100644 index 000000000..2bfb42b65 --- /dev/null +++ b/measure/src/main/scala/org/apache/griffin/measure/rule/func/DefaultFunctionDefine.scala @@ -0,0 +1,29 @@ +/*- + * Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. + + */ +package org.apache.griffin.measure.rule.func + +import org.apache.griffin.measure.utils.JsonUtil + +class DefaultFunctionDefine extends FunctionDefine { + + def json(str: String): Map[String, Any] = { + try { + JsonUtil.toAnyMap(str) + } catch { + case e: Throwable => throw e + } + } + +} diff --git a/measure/src/main/scala/org/apache/griffin/measure/rule/func/FunctionDefine.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/func/FunctionDefine.scala new file mode 100644 index 000000000..860d4c398 --- /dev/null +++ b/measure/src/main/scala/org/apache/griffin/measure/rule/func/FunctionDefine.scala @@ -0,0 +1,21 @@ +/*- + * Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. + + */ +package org.apache.griffin.measure.rule.func + +trait FunctionDefine extends Serializable { + +} + +class UnKnown {} \ No newline at end of file diff --git a/measure/src/main/scala/org/apache/griffin/measure/rule/func/FunctionUtil.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/func/FunctionUtil.scala new file mode 100644 index 000000000..623c8a07d --- /dev/null +++ b/measure/src/main/scala/org/apache/griffin/measure/rule/func/FunctionUtil.scala @@ -0,0 +1,70 @@ +/*- + * Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. + + */ +package org.apache.griffin.measure.rule.func + +import java.lang.reflect.Method + +import org.apache.griffin.measure.log.Loggable + +import scala.collection.mutable.{Map => MutableMap} + +object FunctionUtil extends Loggable { + + val functionDefines: MutableMap[String, FunctionDefine] = MutableMap[String, FunctionDefine]() + + registerFunctionDefine(Array(classOf[DefaultFunctionDefine].getCanonicalName)) + + def registerFunctionDefine(classes: Iterable[String]): Unit = { + for (cls <- classes) { + try { + val clz: Class[_] = Class.forName(cls) + if (clz.isAssignableFrom(classOf[FunctionDefine])) { + functionDefines += (cls -> clz.newInstance.asInstanceOf[FunctionDefine]) + } else { + warn(s"${cls} register fails: ${cls} is not sub class of ${classOf[FunctionDefine].getCanonicalName}") + } + } catch { + case e: Throwable => warn(s"${cls} register fails: ${e.getMessage}") + } + } + } + + def invoke(methodName: String, params: Array[Option[Any]]): Seq[Option[Any]] = { + val paramTypes = params.map { param => + try { + param match { + case Some(v) => v.getClass + case _ => classOf[UnKnown] + } + } catch { + case e: Throwable => classOf[UnKnown] + } + } + + functionDefines.values.foldLeft(Nil: Seq[Option[Any]]) { (res, funcDef) => + if (res.isEmpty) { + val clz = funcDef.getClass + try { + val method = clz.getMethod(methodName, paramTypes: _*) + Seq(Some(method.invoke(funcDef, params: _*))) + } catch { + case e: Throwable => res + } + } else res + } + } + +} + diff --git a/measure/src/main/scala/org/apache/griffin/measure/utils/JsonUtil.scala b/measure/src/main/scala/org/apache/griffin/measure/utils/JsonUtil.scala index aee13d0fb..c5218350f 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/utils/JsonUtil.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/utils/JsonUtil.scala @@ -43,4 +43,8 @@ object JsonUtil { def fromJson[T: ClassTag](is: InputStream)(implicit m : Manifest[T]): T = { mapper.readValue[T](is, classTag[T].runtimeClass.asInstanceOf[Class[T]]) } + + def toAnyMap(json: String) = { + mapper.readValue(json, classOf[Map[String, Any]]) + } } diff --git a/measure/src/main/scala/org/apache/griffin/measure/utils/TimeUtil.scala b/measure/src/main/scala/org/apache/griffin/measure/utils/TimeUtil.scala index d235bd8b2..c916f19ec 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/utils/TimeUtil.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/utils/TimeUtil.scala @@ -42,6 +42,29 @@ object TimeUtil { case Failure(ex) => throw ex } } + value + } + + def timeToUnit(ms: Long, unit: String): Long = { + unit match { + case "ms" => ms + case "sec" => ms / 1000 + case "min" => ms / (60 * 1000) + case "hour" => ms / (60 * 60 * 1000) + case "day" => ms / (24 * 60 * 60 * 1000) + case _ => ms / (60 * 1000) + } + } + + def timeFromUnit(t: Long, unit: String): Long = { + unit match { + case "ms" => t + case "sec" => t * 1000 + case "min" => t * 60 * 1000 + case "hour" => t * 60 * 60 * 1000 + case "day" => t * 24 * 60 * 60 * 1000 + case _ => t * 60 * 1000 + } } } diff --git a/service/src/main/resources/sparkJob.properties b/service/src/main/resources/sparkJob.properties index 51969ac31..819f00d39 100644 --- a/service/src/main/resources/sparkJob.properties +++ b/service/src/main/resources/sparkJob.properties @@ -1,4 +1,4 @@ -sparkJob.file=hdfs:///griffin/griffin-measure.jar +ssparkJob.file=hdfs:///griffin/griffin-measure.jar sparkJob.className=org.apache.griffin.measure.Application sparkJob.args_1=hdfs:///griffin/json/env.json sparkJob.args_3=hdfs,raw From da7cfe213a1aea4deafa538522a1d34f2adce151 Mon Sep 17 00:00:00 2001 From: Lionel Liu Date: Thu, 6 Jul 2017 15:44:48 +0800 Subject: [PATCH 009/111] v4 --- measure/src/main/resources/env.json | 1 + .../algo/batch/BatchAccuracyAlgo.scala | 28 ++++---- .../measure/algo/batch/BatchProfileAlgo.scala | 20 +++--- .../streaming/StreamingAccuracyAlgo.scala | 69 ++++++++----------- .../algo/streaming/StreamingProcess.scala | 42 +++++++++++ .../griffin/measure/cache/InfoCache.scala | 2 +- .../measure/cache/InfoCacheInstance.scala | 2 + .../measure/cache/MultiCacheLock.scala | 35 ++++++++++ .../config/params/env/SparkParam.scala | 1 + .../griffin/measure/rule/expr/Expr.scala | 2 - .../measure/rule/expr/LiteralExpr.scala | 6 -- .../measure/rule/expr/LogicalExpr.scala | 1 - .../griffin/measure/rule/expr/MathExpr.scala | 13 ---- .../measure/rule/expr/SelectExpr.scala | 2 + 14 files changed, 135 insertions(+), 89 deletions(-) create mode 100644 measure/src/main/scala/org/apache/griffin/measure/algo/streaming/StreamingProcess.scala create mode 100644 measure/src/main/scala/org/apache/griffin/measure/cache/MultiCacheLock.scala diff --git a/measure/src/main/resources/env.json b/measure/src/main/resources/env.json index 10250feb6..4dc0ef23c 100644 --- a/measure/src/main/resources/env.json +++ b/measure/src/main/resources/env.json @@ -3,6 +3,7 @@ "log.level": "INFO", "checkpoint.dir": "hdfs:///griffin/batch/cp", "batch.interval": "10s", + "process.interval": "10m", "config": {} }, diff --git a/measure/src/main/scala/org/apache/griffin/measure/algo/batch/BatchAccuracyAlgo.scala b/measure/src/main/scala/org/apache/griffin/measure/algo/batch/BatchAccuracyAlgo.scala index deced368e..7e2e5ae56 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/algo/batch/BatchAccuracyAlgo.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/algo/batch/BatchAccuracyAlgo.scala @@ -66,11 +66,15 @@ case class BatchAccuracyAlgo(allParam: AllParam) extends AccuracyAlgo { // const expr value map val constExprValueMap = ExprValueUtil.genExprValueMaps(None, ruleAnalyzer.constCacheExprs, Map[String, Any]()) val finalConstExprValueMap = ExprValueUtil.updateExprValueMaps(ruleAnalyzer.constFinalCacheExprs, constExprValueMap) + val finalConstMap = finalConstExprValueMap.headOption match { + case Some(m) => m + case _ => Map[String, Any]() + } // data connector val sourceDataConnector: BatchDataConnector = DataConnectorFactory.getBatchDataConnector(sqlContext, userParam.sourceParam, - ruleAnalyzer.sourceRuleExprs, finalConstExprValueMap + ruleAnalyzer.sourceRuleExprs, finalConstMap ) match { case Success(cntr) => { if (cntr.available) cntr @@ -80,7 +84,7 @@ case class BatchAccuracyAlgo(allParam: AllParam) extends AccuracyAlgo { } val targetDataConnector: BatchDataConnector = DataConnectorFactory.getBatchDataConnector(sqlContext, userParam.targetParam, - ruleAnalyzer.targetRuleExprs, finalConstExprValueMap + ruleAnalyzer.targetRuleExprs, finalConstMap ) match { case Success(cntr) => { if (cntr.available) cntr @@ -100,11 +104,11 @@ case class BatchAccuracyAlgo(allParam: AllParam) extends AccuracyAlgo { // } // get data - val sourceData: RDD[(Product, Map[String, Any])] = sourceDataConnector.data() match { + val sourceData: RDD[(Product, (Map[String, Any], Map[String, Any]))] = sourceDataConnector.data() match { case Success(dt) => dt case Failure(ex) => throw ex } - val targetData: RDD[(Product, Map[String, Any])] = targetDataConnector.data() match { + val targetData: RDD[(Product, (Map[String, Any], Map[String, Any]))] = targetDataConnector.data() match { case Success(dt) => dt case Failure(ex) => throw ex } @@ -139,17 +143,13 @@ case class BatchAccuracyAlgo(allParam: AllParam) extends AccuracyAlgo { } // calculate accuracy between source data and target data - def accuracy(sourceData: RDD[(Product, Map[String, Any])], targetData: RDD[(Product, Map[String, Any])], ruleAnalyzer: RuleAnalyzer - ): (AccuracyResult, RDD[(Product, (Map[String, Any], Map[String, Any]))], RDD[(Product, (Map[String, Any], Map[String, Any]))]) = { - - // 1. wrap data - val sourceWrappedData: RDD[(Product, (Map[String, Any], Map[String, Any]))] = sourceData.map(r => (r._1, wrapInitData(r._2))) - val targetWrappedData: RDD[(Product, (Map[String, Any], Map[String, Any]))] = targetData.map(r => (r._1, wrapInitData(r._2))) - - // 2. cogroup - val allKvs = sourceWrappedData.cogroup(targetWrappedData) + def accuracy(sourceData: RDD[(Product, (Map[String, Any], Map[String, Any]))], + targetData: RDD[(Product, (Map[String, Any], Map[String, Any]))], + ruleAnalyzer: RuleAnalyzer) = { + // 1. cogroup + val allKvs = sourceData.cogroup(targetData) - // 3. accuracy calculation + // 2. accuracy calculation val (accuResult, missingRdd, matchedRdd) = AccuracyCore.accuracy(allKvs, ruleAnalyzer) (accuResult, missingRdd, matchedRdd) diff --git a/measure/src/main/scala/org/apache/griffin/measure/algo/batch/BatchProfileAlgo.scala b/measure/src/main/scala/org/apache/griffin/measure/algo/batch/BatchProfileAlgo.scala index 21af67a40..dd040f83f 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/algo/batch/BatchProfileAlgo.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/algo/batch/BatchProfileAlgo.scala @@ -66,11 +66,15 @@ case class BatchProfileAlgo(allParam: AllParam) extends ProfileAlgo { // const expr value map val constExprValueMap = ExprValueUtil.genExprValueMaps(None, ruleAnalyzer.constCacheExprs, Map[String, Any]()) val finalConstExprValueMap = ExprValueUtil.updateExprValueMaps(ruleAnalyzer.constFinalCacheExprs, constExprValueMap) + val finalConstMap = finalConstExprValueMap.headOption match { + case Some(m) => m + case _ => Map[String, Any]() + } // data connector val sourceDataConnector: BatchDataConnector = DataConnectorFactory.getBatchDataConnector(sqlContext, userParam.sourceParam, - ruleAnalyzer.sourceRuleExprs, finalConstExprValueMap + ruleAnalyzer.sourceRuleExprs, finalConstMap ) match { case Success(cntr) => { if (cntr.available) cntr @@ -86,7 +90,7 @@ case class BatchProfileAlgo(allParam: AllParam) extends ProfileAlgo { // } // get data - val sourceData: RDD[(Product, Map[String, Any])] = sourceDataConnector.data() match { + val sourceData: RDD[(Product, (Map[String, Any], Map[String, Any]))] = sourceDataConnector.data() match { case Success(dt) => dt case Failure(ex) => throw ex } @@ -120,14 +124,10 @@ case class BatchProfileAlgo(allParam: AllParam) extends ProfileAlgo { } // calculate profile from source data - def profile(sourceData: RDD[(Product, Map[String, Any])], ruleAnalyzer: RuleAnalyzer - ): (ProfileResult, RDD[(Product, (Map[String, Any], Map[String, Any]))], RDD[(Product, (Map[String, Any], Map[String, Any]))]) = { - - // 1. wrap data - val sourceWrappedData: RDD[(Product, (Map[String, Any], Map[String, Any]))] = sourceData.map(r => (r._1, wrapInitData(r._2))) - - // 2. profile calculation - val (profileResult, missingRdd, matchedRdd) = ProfileCore.profile(sourceWrappedData, ruleAnalyzer) + def profile(sourceData: RDD[(Product, (Map[String, Any], Map[String, Any]))], ruleAnalyzer: RuleAnalyzer + ) = { + // 1. profile calculation + val (profileResult, missingRdd, matchedRdd) = ProfileCore.profile(sourceData, ruleAnalyzer) (profileResult, missingRdd, matchedRdd) } diff --git a/measure/src/main/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgo.scala b/measure/src/main/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgo.scala index 5ed81c63e..c132c5fe2 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgo.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgo.scala @@ -15,7 +15,7 @@ limitations under the License. package org.apache.griffin.measure.algo.streaming import java.util.Date -import java.util.concurrent.TimeUnit +import java.util.concurrent.{Executors, ThreadPoolExecutor, TimeUnit} import org.apache.griffin.measure.algo.AccuracyAlgo import org.apache.griffin.measure.algo.core.AccuracyCore @@ -50,11 +50,11 @@ case class StreamingAccuracyAlgo(allParam: AllParam) extends AccuracyAlgo { sc.setLogLevel(sparkParam.logLevel) val sqlContext = new HiveContext(sc) - val interval = TimeUtil.milliseconds(sparkParam.batchInterval) match { + val batchInterval = TimeUtil.milliseconds(sparkParam.batchInterval) match { case Some(interval) => Milliseconds(interval) case _ => throw new Exception("invalid batch interval") } - val ssc = new StreamingContext(sc, interval) + val ssc = new StreamingContext(sc, batchInterval) ssc.checkpoint(sparkParam.cpDir) // start time @@ -77,11 +77,15 @@ case class StreamingAccuracyAlgo(allParam: AllParam) extends AccuracyAlgo { // const expr value map val constExprValueMap = ExprValueUtil.genExprValueMaps(None, ruleAnalyzer.constCacheExprs, Map[String, Any]()) val finalConstExprValueMap = ExprValueUtil.updateExprValueMaps(ruleAnalyzer.constFinalCacheExprs, constExprValueMap) + val finalConstMap = finalConstExprValueMap.headOption match { + case Some(m) => m + case _ => Map[String, Any]() + } // data connector val sourceDataConnector: BatchDataConnector = DataConnectorFactory.getDataConnector(sqlContext, ssc, userParam.sourceParam, - ruleAnalyzer.sourceRuleExprs, finalConstExprValueMap.head + ruleAnalyzer.sourceRuleExprs, finalConstMap ) match { case Success(cntr) => { if (cntr.available) cntr @@ -91,7 +95,7 @@ case class StreamingAccuracyAlgo(allParam: AllParam) extends AccuracyAlgo { } val targetDataConnector: BatchDataConnector = DataConnectorFactory.getDataConnector(sqlContext, ssc, userParam.targetParam, - ruleAnalyzer.targetRuleExprs, finalConstExprValueMap.head + ruleAnalyzer.targetRuleExprs, finalConstMap ) match { case Success(cntr) => { if (cntr.available) cntr @@ -104,8 +108,8 @@ case class StreamingAccuracyAlgo(allParam: AllParam) extends AccuracyAlgo { InfoCacheInstance.init // process thread - case class Process() extends Runnable { - val lock = InfoCacheInstance.genLock + case class Process(name: String) extends Runnable { + val lock = InfoCacheInstance.genLock(name) def run(): Unit = { val locked = lock.lock(5, TimeUnit.SECONDS) if (locked) { @@ -124,18 +128,20 @@ case class StreamingAccuracyAlgo(allParam: AllParam) extends AccuracyAlgo { // accuracy algorithm val (accuResult, missingRdd, matchedRdd) = accuracy(sourceData, targetData, ruleAnalyzer) + println(accuResult) + val et = new Date().getTime // persist time - persist.log(et, s"calculation using time: ${et - st} ms") +// persist.log(et, s"calculation using time: ${et - st} ms") // persist result - persist.result(et, accuResult) +// persist.result(et, accuResult) val missingRecords = missingRdd.map(record2String(_, ruleAnalyzer.sourceRuleExprs.persistExprs, ruleAnalyzer.targetRuleExprs.persistExprs)) - persist.missRecords(missingRecords) +// persist.missRecords(missingRecords) - val pet = new Date().getTime - persist.log(pet, s"persist using time: ${pet - et} ms") +// val pet = new Date().getTime +// persist.log(pet, s"persist using time: ${pet - et} ms") } finally { lock.unlock() } @@ -143,36 +149,13 @@ case class StreamingAccuracyAlgo(allParam: AllParam) extends AccuracyAlgo { } } - // fixme: thread pool - - // get data -// val sourceData: RDD[(Product, Map[String, Any])] = sourceDataConnector.data() match { -// case Success(dt) => dt -// case Failure(ex) => throw ex -// } -// val targetData: RDD[(Product, Map[String, Any])] = targetDataConnector.data() match { -// case Success(dt) => dt -// case Failure(ex) => throw ex -// } -// -// // accuracy algorithm -// val (accuResult, missingRdd, matchedRdd) = accuracy(sourceData, targetData, ruleAnalyzer) -// -// // end time -// val endTime = new Date().getTime -// persist.log(endTime, s"calculation using time: ${endTime - startTime} ms") -// -// // persist result -// persist.result(endTime, accuResult) -// val missingRecords = missingRdd.map(record2String(_, ruleAnalyzer.sourceRuleExprs.persistExprs, ruleAnalyzer.targetRuleExprs.persistExprs)) -// persist.missRecords(missingRecords) -// -// // persist end time -// val persistEndTime = new Date().getTime -// persist.log(persistEndTime, s"persist using time: ${persistEndTime - endTime} ms") -// -// // finish -// persist.finish() + val processInterval = TimeUtil.milliseconds(sparkParam.processInterval) match { + case Some(interval) => interval + case _ => throw new Exception("invalid batch interval") + } + val process = StreamingProcess(processInterval, Process(metricName)) + + process.startup() ssc.start() ssc.awaitTermination() @@ -184,6 +167,8 @@ case class StreamingAccuracyAlgo(allParam: AllParam) extends AccuracyAlgo { InfoCacheInstance.close persist.finish() + + process.shutdown() } } diff --git a/measure/src/main/scala/org/apache/griffin/measure/algo/streaming/StreamingProcess.scala b/measure/src/main/scala/org/apache/griffin/measure/algo/streaming/StreamingProcess.scala new file mode 100644 index 000000000..807795dcf --- /dev/null +++ b/measure/src/main/scala/org/apache/griffin/measure/algo/streaming/StreamingProcess.scala @@ -0,0 +1,42 @@ +/*- + * Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. + + */ +package org.apache.griffin.measure.algo.streaming + +import java.util.concurrent.{Executors, ThreadPoolExecutor, TimeUnit} +import java.util.{Timer, TimerTask} + +case class StreamingProcess(interval: Long, runnable: Runnable) { + + val pool: ThreadPoolExecutor = Executors.newFixedThreadPool(5).asInstanceOf[ThreadPoolExecutor] + + val timer = new Timer("process", true) + + val timerTask = new TimerTask() { + override def run(): Unit = { + pool.submit(runnable) + } + } + + def startup(): Unit = { + timer.schedule(timerTask, 0, interval) + } + + def shutdown(): Unit = { + timer.cancel() + pool.shutdown() + pool.awaitTermination(10, TimeUnit.SECONDS) + } + +} diff --git a/measure/src/main/scala/org/apache/griffin/measure/cache/InfoCache.scala b/measure/src/main/scala/org/apache/griffin/measure/cache/InfoCache.scala index 72496b8ba..a1ea09a74 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/cache/InfoCache.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/cache/InfoCache.scala @@ -27,6 +27,6 @@ trait InfoCache extends Loggable with Serializable { def deleteInfo(keys: Iterable[String]): Unit def clearInfo(): Unit - def genLock(): CacheLock + def genLock(s: String): CacheLock } diff --git a/measure/src/main/scala/org/apache/griffin/measure/cache/InfoCacheInstance.scala b/measure/src/main/scala/org/apache/griffin/measure/cache/InfoCacheInstance.scala index 65ae24889..779fb9a27 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/cache/InfoCacheInstance.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/cache/InfoCacheInstance.scala @@ -37,4 +37,6 @@ object InfoCacheInstance extends InfoCache { } def deleteInfo(keys: Iterable[String]): Unit = infoCaches.foreach(_.deleteInfo(keys)) def clearInfo(): Unit = infoCaches.foreach(_.clearInfo) + + def genLock(s: String): CacheLock = MultiCacheLock(infoCaches.map(_.genLock(s))) } \ No newline at end of file diff --git a/measure/src/main/scala/org/apache/griffin/measure/cache/MultiCacheLock.scala b/measure/src/main/scala/org/apache/griffin/measure/cache/MultiCacheLock.scala new file mode 100644 index 000000000..0a0a5859d --- /dev/null +++ b/measure/src/main/scala/org/apache/griffin/measure/cache/MultiCacheLock.scala @@ -0,0 +1,35 @@ +/*- + * Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. + + */ +package org.apache.griffin.measure.cache + +import java.util.concurrent.TimeUnit + +case class MultiCacheLock(cacheLocks: List[CacheLock]) extends CacheLock { + + def lock(outtime: Long, unit: TimeUnit): Boolean = { + cacheLocks.headOption match { + case Some(cl) => cl.lock(outtime, unit) + case None => true + } + } + + def unlock(): Unit = { + cacheLocks.headOption match { + case Some(cl) => cl.unlock + case None => {} + } + } + +} diff --git a/measure/src/main/scala/org/apache/griffin/measure/config/params/env/SparkParam.scala b/measure/src/main/scala/org/apache/griffin/measure/config/params/env/SparkParam.scala index d6fb3647a..1927f81a9 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/config/params/env/SparkParam.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/config/params/env/SparkParam.scala @@ -22,6 +22,7 @@ import org.apache.griffin.measure.config.params.Param case class SparkParam( @JsonProperty("log.level") logLevel: String, @JsonProperty("checkpoint.dir") cpDir: String, @JsonProperty("batch.interval") batchInterval: String, + @JsonProperty("process.interval") processInterval: String, @JsonProperty("config") config: Map[String, Any] ) extends Param { diff --git a/measure/src/main/scala/org/apache/griffin/measure/rule/expr/Expr.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/expr/Expr.scala index 805f2b9ab..8e67cd9e0 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rule/expr/Expr.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rule/expr/Expr.scala @@ -22,8 +22,6 @@ trait Expr extends Serializable with Describable with Cacheable with Calculatabl val _id = ExprIdCounter.genId(_defaultId) - def dataType: DataType - protected def getSubCacheExprs(ds: String): Iterable[Expr] = Nil final def getCacheExprs(ds: String): Iterable[Expr] = { if (cacheable(ds)) getSubCacheExprs(ds).toList :+ this else getSubCacheExprs(ds) diff --git a/measure/src/main/scala/org/apache/griffin/measure/rule/expr/LiteralExpr.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/expr/LiteralExpr.scala index 9dfd295de..ecc9fd26b 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rule/expr/LiteralExpr.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rule/expr/LiteralExpr.scala @@ -28,7 +28,6 @@ trait LiteralExpr extends Expr { case class LiteralStringExpr(expr: String) extends LiteralExpr { val value: Option[String] = Some(expr) val desc: String = s"'${value.getOrElse("")}'" - def dataType: DataType = StringType } case class LiteralNumberExpr(expr: String) extends LiteralExpr { @@ -46,14 +45,12 @@ case class LiteralNumberExpr(expr: String) extends LiteralExpr { } } val desc: String = value.getOrElse("").toString - def dataType: DataType = DoubleType } case class LiteralTimeExpr(expr: String) extends LiteralExpr { final val TimeRegex = """(\d+)(d|h|m|s|ms)""".r val value: Option[Long] = TimeUtil.milliseconds(expr) val desc: String = expr - def dataType: DataType = LongType } case class LiteralBooleanExpr(expr: String) extends LiteralExpr { @@ -65,17 +62,14 @@ case class LiteralBooleanExpr(expr: String) extends LiteralExpr { case _ => throw new Exception(s"${expr} is invalid boolean") } val desc: String = value.getOrElse("").toString - def dataType: DataType = BooleanType } case class LiteralNullExpr(expr: String) extends LiteralExpr { val value: Option[Any] = Some(null) val desc: String = "null" - def dataType: DataType = NullType } case class LiteralNoneExpr(expr: String) extends LiteralExpr { val value: Option[Any] = None val desc: String = "none" - def dataType: DataType = NullType } \ No newline at end of file diff --git a/measure/src/main/scala/org/apache/griffin/measure/rule/expr/LogicalExpr.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/expr/LogicalExpr.scala index ca99fb2b2..e9fa0da7e 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rule/expr/LogicalExpr.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rule/expr/LogicalExpr.scala @@ -19,7 +19,6 @@ import org.apache.spark.sql.types.{BooleanType, DataType} trait LogicalExpr extends Expr with AnalyzableExpr { override def cacheUnit: Boolean = true - def dataType: DataType = BooleanType } case class LogicalSimpleExpr(expr: MathExpr) extends LogicalExpr { diff --git a/measure/src/main/scala/org/apache/griffin/measure/rule/expr/MathExpr.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/expr/MathExpr.scala index 5cc225b24..fc60b02a5 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rule/expr/MathExpr.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rule/expr/MathExpr.scala @@ -24,7 +24,6 @@ trait MathExpr extends Expr { case class MathFactorExpr(self: Expr) extends MathExpr { def calculateOnly(values: Map[String, Any]): Option[Any] = self.calculate(values) - def dataType: DataType = self.dataType val desc: String = self.desc val dataSources: Set[String] = self.dataSources override def getSubCacheExprs(ds: String): Iterable[Expr] = { @@ -50,11 +49,6 @@ case class UnaryMathExpr(oprList: Iterable[String], factor: Expr) extends MathEx } } } - def dataType: DataType = { - oprList.foldRight(factor.dataType unaryOpr) { (opr, tp) => - tp unaryOpr - } - } val desc: String = oprList.foldRight(factor.desc) { (prev, ex) => s"${prev}${ex}" } val dataSources: Set[String] = factor.dataSources override def cacheUnit: Boolean = true @@ -86,13 +80,6 @@ case class BinaryMathExpr(first: MathExpr, others: Iterable[(String, MathExpr)]) } } } - def dataType: DataType = { - others.foldLeft(first.dataType) { (tp, pair) => - val (opr, next) = pair - val ntp = next.dataType - tp binaryOpr ntp - } - } val desc: String = others.foldLeft(first.desc) { (ex, next) => s"${ex} ${next._1} ${next._2.desc}" } val dataSources: Set[String] = first.dataSources ++ others.flatMap(_._2.dataSources).toSet override def cacheUnit: Boolean = true diff --git a/measure/src/main/scala/org/apache/griffin/measure/rule/expr/SelectExpr.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/expr/SelectExpr.scala index 04bad4247..38ac8b8b9 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rule/expr/SelectExpr.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rule/expr/SelectExpr.scala @@ -14,6 +14,8 @@ limitations under the License. */ package org.apache.griffin.measure.rule.expr +import org.apache.spark.sql.types.DataType + trait SelectExpr extends Expr { def calculateOnly(values: Map[String, Any]): Option[Any] = None } From 84d55c912d80791715c5b4a09b510bf9d4c51215 Mon Sep 17 00:00:00 2001 From: Lionel Liu Date: Thu, 6 Jul 2017 18:34:29 +0800 Subject: [PATCH 010/111] t1 --- measure/src/main/resources/env.json | 6 +- .../streaming/StreamingAccuracyAlgo.scala | 6 +- .../griffin/measure/cache/ZKCacheLock.scala | 2 +- .../griffin/measure/cache/ZKInfoCache.scala | 26 +++--- .../measure/cache/ZKCacheLockTest.scala | 79 +++++++++++++++++ .../measure/cache/ZKInfoCacheTest.scala | 85 +++++++++++++++++++ .../measure/connector/ConnectorTest.scala | 14 +++ 7 files changed, 196 insertions(+), 22 deletions(-) create mode 100644 measure/src/test/scala/org/apache/griffin/measure/cache/ZKCacheLockTest.scala create mode 100644 measure/src/test/scala/org/apache/griffin/measure/cache/ZKInfoCacheTest.scala diff --git a/measure/src/main/resources/env.json b/measure/src/main/resources/env.json index 4dc0ef23c..14e3b75a2 100644 --- a/measure/src/main/resources/env.json +++ b/measure/src/main/resources/env.json @@ -30,11 +30,11 @@ "type": "zk", "config": { "hosts": "localhost:2181", - "parent.path": "/griffin/cache", + "namespace": "griffin/infocache", + "lock.path": "lock", "mode": "persist", "init.clear": true, - "close.clear": false, - "lock.path": "lock" + "close.clear": false } } ], diff --git a/measure/src/main/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgo.scala b/measure/src/main/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgo.scala index c132c5fe2..0c7386f87 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgo.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgo.scala @@ -108,8 +108,8 @@ case class StreamingAccuracyAlgo(allParam: AllParam) extends AccuracyAlgo { InfoCacheInstance.init // process thread - case class Process(name: String) extends Runnable { - val lock = InfoCacheInstance.genLock(name) + case class Process() extends Runnable { + val lock = InfoCacheInstance.genLock("process") def run(): Unit = { val locked = lock.lock(5, TimeUnit.SECONDS) if (locked) { @@ -153,7 +153,7 @@ case class StreamingAccuracyAlgo(allParam: AllParam) extends AccuracyAlgo { case Some(interval) => interval case _ => throw new Exception("invalid batch interval") } - val process = StreamingProcess(processInterval, Process(metricName)) + val process = StreamingProcess(processInterval, Process()) process.startup() diff --git a/measure/src/main/scala/org/apache/griffin/measure/cache/ZKCacheLock.scala b/measure/src/main/scala/org/apache/griffin/measure/cache/ZKCacheLock.scala index ccd732b85..d2d829404 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/cache/ZKCacheLock.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/cache/ZKCacheLock.scala @@ -34,7 +34,7 @@ case class ZKCacheLock(@transient mutex: InterProcessMutex) extends CacheLock { def unlock(): Unit = { try { - mutex.release + if (mutex.isAcquiredInThisProcess) mutex.release } catch { case e: Throwable => { error(s"unlock error: ${e.getMessage}") diff --git a/measure/src/main/scala/org/apache/griffin/measure/cache/ZKInfoCache.scala b/measure/src/main/scala/org/apache/griffin/measure/cache/ZKInfoCache.scala index 9d7886869..33d7acda9 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/cache/ZKInfoCache.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/cache/ZKInfoCache.scala @@ -23,7 +23,7 @@ import org.apache.zookeeper.CreateMode case class ZKInfoCache(config: Map[String, Any], metricName: String) extends InfoCache { val Hosts = "hosts" - val ParentPath = "parent.path" + val Namespace = "namespace" val Mode = "mode" val InitClear = "init.clear" val CloseClear = "close.clear" @@ -35,7 +35,7 @@ case class ZKInfoCache(config: Map[String, Any], metricName: String) extends Inf final val separator = "/" val hosts = config.getOrElse(Hosts, "").toString - val parentPath = config.getOrElse(ParentPath, separator).toString + val namespace = config.getOrElse(Namespace, "").toString val mode: CreateMode = config.get(Mode) match { case Some(s: String) => s match { case PersistRegex() => CreateMode.PERSISTENT @@ -54,11 +54,16 @@ case class ZKInfoCache(config: Map[String, Any], metricName: String) extends Inf } val lockPath = config.getOrElse(LockPath, "lock").toString - private val client: CuratorFramework = CuratorFrameworkFactory.newClient( - Hosts, new ExponentialBackoffRetry(1000, 3)).usingNamespace(cacheParentPath) + private val cacheNamespace: String = if (namespace.isEmpty) metricName else namespace + separator + metricName + private val builder = CuratorFrameworkFactory.builder() + .connectString("localhost:2181") + .retryPolicy(new ExponentialBackoffRetry(1000, 3)) + .namespace(cacheNamespace) + private val client: CuratorFramework = builder.build def init(): Unit = { client.start() + client.usingNamespace(cacheNamespace) if (initClear) { clearInfo } @@ -87,7 +92,7 @@ case class ZKInfoCache(config: Map[String, Any], metricName: String) extends Inf def readInfo(keys: Iterable[String]): Map[String, String] = { keys.flatMap { key => - read(key) match { + read(path(key)) match { case Some(v) => Some((key, v)) case _ => None } @@ -103,7 +108,7 @@ case class ZKInfoCache(config: Map[String, Any], metricName: String) extends Inf } def genLock(s: String): ZKCacheLock = { - val lpt = if (s.isEmpty) path(lockPath) else validHeadPath(lockPath) + s + val lpt = if (s.isEmpty) path(lockPath) else path(lockPath) + separator + s ZKCacheLock(new InterProcessMutex(client, lpt)) } @@ -111,15 +116,6 @@ case class ZKInfoCache(config: Map[String, Any], metricName: String) extends Inf if (k.startsWith(separator)) k else separator + k } - private def validHeadPath(head: String): String = { - val hd = if (head.startsWith(separator)) head else separator + head - if (hd.endsWith(separator)) hd else hd + separator - } - - private def cacheParentPath: String = { - validHeadPath(parentPath) + metricName - } - private def createOrUpdate(path: String, content: String): Boolean = { if (checkExists(path)) { update(path, content) diff --git a/measure/src/test/scala/org/apache/griffin/measure/cache/ZKCacheLockTest.scala b/measure/src/test/scala/org/apache/griffin/measure/cache/ZKCacheLockTest.scala new file mode 100644 index 000000000..f016ac6e1 --- /dev/null +++ b/measure/src/test/scala/org/apache/griffin/measure/cache/ZKCacheLockTest.scala @@ -0,0 +1,79 @@ +/*- + * Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. + + */ +package org.apache.griffin.measure.cache + +import java.util.Date +import java.util.concurrent.{Executors, ThreadPoolExecutor, TimeUnit} + +import org.apache.curator.framework.recipes.locks.InterProcessMutex +import org.apache.curator.framework.{CuratorFramework, CuratorFrameworkFactory} +import org.apache.curator.retry.ExponentialBackoffRetry +import org.junit.runner.RunWith +import org.scalatest.junit.JUnitRunner +import org.scalatest.{BeforeAndAfter, FunSuite, Matchers} + +import scala.util.{Failure, Try} + +@RunWith(classOf[JUnitRunner]) +class ZKCacheLockTest extends FunSuite with Matchers with BeforeAndAfter { + + val map = Map[String, Any]( + ("hosts" -> "localhost:2181"), + ("namespace" -> "griffin/infocache"), + ("lock.path" -> "lock"), + ("mode" -> "persist"), + ("init.clear" -> true), + ("close.clear" -> false) + ) + val name = "ttt" + + val ic = ZKInfoCache(map, name) + + before { + ic.init + } + + test ("lock") { + + case class Proc(n: Int) extends Runnable { + override def run(): Unit = { + val cl = ic.genLock("proc") + val b = cl.lock(2, TimeUnit.SECONDS) + try { + println(s"${n}: ${b}") + if (b) Thread.sleep(3000) + } finally { + cl.unlock() + } + } + } + + val pool = Executors.newFixedThreadPool(5).asInstanceOf[ThreadPoolExecutor] + val t = 0 until 10 + t.foreach(a => pool.submit(Proc(a))) + + pool.shutdown() + val t1 = new Date() + println(s"${t1}: pool shut down") + pool.awaitTermination(20, TimeUnit.SECONDS) + val t2 = new Date() + println(s"${t2}: pool shut down done [${t2.getTime - t1.getTime}]") + } + + after { + ic.close() + } + +} diff --git a/measure/src/test/scala/org/apache/griffin/measure/cache/ZKInfoCacheTest.scala b/measure/src/test/scala/org/apache/griffin/measure/cache/ZKInfoCacheTest.scala new file mode 100644 index 000000000..f1a4c382b --- /dev/null +++ b/measure/src/test/scala/org/apache/griffin/measure/cache/ZKInfoCacheTest.scala @@ -0,0 +1,85 @@ +/*- + * Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. + + */ +package org.apache.griffin.measure.cache + +import java.util.Date +import java.util.concurrent.{Executors, ThreadPoolExecutor, TimeUnit} + +import org.apache.curator.framework.recipes.locks.InterProcessMutex +import org.apache.curator.framework.{CuratorFramework, CuratorFrameworkFactory} +import org.apache.curator.retry.ExponentialBackoffRetry +import org.junit.runner.RunWith +import org.scalatest.junit.JUnitRunner +import org.scalatest.{BeforeAndAfter, FunSuite, Matchers} + +import scala.util.{Failure, Try} + +@RunWith(classOf[JUnitRunner]) +class ZKInfoCacheTest extends FunSuite with Matchers with BeforeAndAfter { + + val map = Map[String, Any]( + ("hosts" -> "localhost:2181"), + ("namespace" -> "griffin/infocache"), + ("lock.path" -> "lock"), + ("mode" -> "persist"), + ("init.clear" -> true), + ("close.clear" -> false) + ) + val name = "ttt" + + test ("available") { + val ic = ZKInfoCache(map, name) + ic.init + + ic.available should be (true) + + ic.close + } + + test ("cacheInfo and readInfo") { + val ic = ZKInfoCache(map, name) + ic.init + + val keys = List[String]( + "key1", "key2" + ) + val info = Map[String, String]( + ("key1" -> "value1"), + ("key2" -> "value2") + ) + + ic.cacheInfo(info) should be (true) + ic.readInfo(keys) should be (info) + ic.deleteInfo(keys) + ic.readInfo(keys) should be (Map[String, String]()) + + ic.close + } + + test ("genLock") { + val ic = ZKInfoCache(map, name) + ic.init + + val lock1 = ic.genLock("ttt") + val lock2 = ic.genLock("ttt") + lock1.lock(5, TimeUnit.SECONDS) + lock2.lock(5, TimeUnit.SECONDS) + lock1.unlock + lock2.unlock + + ic.close + } + +} diff --git a/measure/src/test/scala/org/apache/griffin/measure/connector/ConnectorTest.scala b/measure/src/test/scala/org/apache/griffin/measure/connector/ConnectorTest.scala index de56d2c2e..d4f83e09c 100644 --- a/measure/src/test/scala/org/apache/griffin/measure/connector/ConnectorTest.scala +++ b/measure/src/test/scala/org/apache/griffin/measure/connector/ConnectorTest.scala @@ -1,3 +1,17 @@ +/*- + * Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. + + */ package org.apache.griffin.measure.connector import org.apache.griffin.measure.config.params.env._ From 8a95c4a4d018d7f5d80642a6892249606482d029 Mon Sep 17 00:00:00 2001 From: Lionel Liu Date: Fri, 7 Jul 2017 15:17:15 +0800 Subject: [PATCH 011/111] t2 --- .../measure/algo/batch/BatchProfileAlgo.scala | 2 +- .../streaming/StreamingAccuracyAlgo.scala | 2 +- .../measure/cache/InfoCacheFactory.scala | 2 +- .../griffin/measure/rule/RuleAnalyzer.scala | 9 +- .../griffin/measure/rule/RuleParser.scala | 9 +- .../measure/rule/expr/AnalyzableExpr.scala | 1 - .../measure/rule/expr/ClauseExpr.scala | 105 +++++ .../measure/rule/expr/StatementExpr.scala | 66 ---- .../measure/algo/BatchAccuracyAlgoTest.scala | 373 +++++++++--------- .../measure/cache/InfoCacheInstanceTest.scala | 73 ++++ .../measure/rule/RuleAnalyzerTest.scala | 4 - 11 files changed, 379 insertions(+), 267 deletions(-) create mode 100644 measure/src/main/scala/org/apache/griffin/measure/rule/expr/ClauseExpr.scala delete mode 100644 measure/src/main/scala/org/apache/griffin/measure/rule/expr/StatementExpr.scala create mode 100644 measure/src/test/scala/org/apache/griffin/measure/cache/InfoCacheInstanceTest.scala diff --git a/measure/src/main/scala/org/apache/griffin/measure/algo/batch/BatchProfileAlgo.scala b/measure/src/main/scala/org/apache/griffin/measure/algo/batch/BatchProfileAlgo.scala index dd040f83f..a5a89b19a 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/algo/batch/BatchProfileAlgo.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/algo/batch/BatchProfileAlgo.scala @@ -22,7 +22,7 @@ import org.apache.griffin.measure.config.params._ import org.apache.griffin.measure.connector._ import org.apache.griffin.measure.persist.{Persist, PersistFactory} import org.apache.griffin.measure.result._ -import org.apache.griffin.measure.rule.expr.{Expr, StatementExpr} +import org.apache.griffin.measure.rule.expr._ import org.apache.griffin.measure.rule.{ExprValueUtil, RuleAnalyzer, RuleFactory} import org.apache.spark.rdd.RDD import org.apache.spark.sql.hive.HiveContext diff --git a/measure/src/main/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgo.scala b/measure/src/main/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgo.scala index 0c7386f87..cc87f25fe 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgo.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgo.scala @@ -25,7 +25,7 @@ import org.apache.griffin.measure.connector._ import org.apache.griffin.measure.persist.{Persist, PersistFactory} import org.apache.griffin.measure.result.AccuracyResult import org.apache.griffin.measure.rule.{ExprValueUtil, RuleAnalyzer, RuleFactory} -import org.apache.griffin.measure.rule.expr.{Expr, StatementExpr} +import org.apache.griffin.measure.rule.expr._ import org.apache.griffin.measure.utils.TimeUtil import org.apache.spark.rdd.RDD import org.apache.spark.sql.hive.HiveContext diff --git a/measure/src/main/scala/org/apache/griffin/measure/cache/InfoCacheFactory.scala b/measure/src/main/scala/org/apache/griffin/measure/cache/InfoCacheFactory.scala index 9cce29dde..04d97241e 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/cache/InfoCacheFactory.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/cache/InfoCacheFactory.scala @@ -15,7 +15,7 @@ case class InfoCacheFactory(infoCacheParams: Iterable[InfoCacheParam], metricNam case _ => throw new Exception("not supported info cache type") } infoCacheTry match { - case Success(infoCache) if (infoCache.available) => Some(infoCache) + case Success(infoCache) => Some(infoCache) case _ => None } } diff --git a/measure/src/main/scala/org/apache/griffin/measure/rule/RuleAnalyzer.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/RuleAnalyzer.scala index f791de302..00e059480 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rule/RuleAnalyzer.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rule/RuleAnalyzer.scala @@ -37,12 +37,10 @@ case class RuleAnalyzer(rule: StatementExpr) extends Serializable { private val sourceGroupbyExprs: Seq[Expr] = groupbyExprPairs.map(_._1) private val targetGroupbyExprs: Seq[Expr] = groupbyExprPairs.map(_._2) - private val whenClauseExprOpt: Option[LogicalExpr] = rule.getWhenClauseExpr - val sourceRuleExprs: RuleExprs = RuleExprs(sourceGroupbyExprs, sourceCacheExprs, - sourceFinalCacheExprs, sourcePersistExprs, whenClauseExprOpt) + sourceFinalCacheExprs, sourcePersistExprs) val targetRuleExprs: RuleExprs = RuleExprs(targetGroupbyExprs, targetCacheExprs, - targetFinalCacheExprs, targetPersistExprs, whenClauseExprOpt) + targetFinalCacheExprs, targetPersistExprs) } @@ -61,8 +59,7 @@ case class RuleAnalyzer(rule: StatementExpr) extends Serializable { case class RuleExprs(groupbyExprs: Seq[Expr], cacheExprs: Iterable[Expr], finalCacheExprs: Iterable[Expr], - persistExprs: Iterable[Expr], - whenClauseExprOpt: Option[LogicalExpr] + persistExprs: Iterable[Expr] ) { // for example: for a rule "$source.name = $target.name AND $source.age < $target.age + (3 * 4)" // in this rule, for the target data source, the targetRuleExprs looks like below diff --git a/measure/src/main/scala/org/apache/griffin/measure/rule/RuleParser.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/RuleParser.scala index aa419cee1..cedc8350f 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rule/RuleParser.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rule/RuleParser.scala @@ -226,11 +226,14 @@ case class RuleParser() extends JavaTokenParsers with Serializable { // ::= [NOT] [(AND | OR) ]+ | "(" ")" def logicalStatement: Parser[LogicalExpr] = orLogicalStatement + // -- clause statement -- + def whereClause: Parser[WhereClauseExpr] = logicalStatement ^^ { WhereClauseExpr(_) } + def whenClause: Parser[WhenClauseExpr] = WhenKeywords ~> logicalStatement ^^ { WhenClauseExpr(_) } + // -- rule -- // ::= [WHEN ] - def rule: Parser[StatementExpr] = logicalStatement ~ opt(WhenKeywords ~> logicalStatement) ^^ { - case ls ~ Some(ws) => WhenClauseStatementExpr(ls, ws) - case ls ~ _ => SimpleStatementExpr(ls) + def rule: Parser[StatementExpr] = whereClause ~ opt(whenClause) ^^ { + case a ~ b => StatementExpr(a, b) } } diff --git a/measure/src/main/scala/org/apache/griffin/measure/rule/expr/AnalyzableExpr.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/expr/AnalyzableExpr.scala index c9bc58e22..a2dd834fc 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rule/expr/AnalyzableExpr.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rule/expr/AnalyzableExpr.scala @@ -17,5 +17,4 @@ package org.apache.griffin.measure.rule.expr trait AnalyzableExpr extends Serializable { def getGroupbyExprPairs(dsPair: (String, String)): Seq[(Expr, Expr)] = Nil - def getWhenClauseExpr(): Option[LogicalExpr] = None } \ No newline at end of file diff --git a/measure/src/main/scala/org/apache/griffin/measure/rule/expr/ClauseExpr.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/expr/ClauseExpr.scala new file mode 100644 index 000000000..7c412c2dd --- /dev/null +++ b/measure/src/main/scala/org/apache/griffin/measure/rule/expr/ClauseExpr.scala @@ -0,0 +1,105 @@ +/*- + * Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. + + */ +package org.apache.griffin.measure.rule.expr + + +trait ClauseExpr extends Expr with AnalyzableExpr { + def valid(values: Map[String, Any]): Boolean = true + override def cacheUnit: Boolean = true +} + +case class WhereClauseExpr(expr: LogicalExpr) extends ClauseExpr { + def calculateOnly(values: Map[String, Any]): Option[Any] = expr.calculate(values) + val desc: String = expr.desc + val dataSources: Set[String] = expr.dataSources + override def getSubCacheExprs(ds: String): Iterable[Expr] = { + expr.getCacheExprs(ds) + } + override def getSubFinalCacheExprs(ds: String): Iterable[Expr] = { + expr.getFinalCacheExprs(ds) + } + override def getSubPersistExprs(ds: String): Iterable[Expr] = { + expr.getPersistExprs(ds) + } + + override def getGroupbyExprPairs(dsPair: (String, String)): Seq[(Expr, Expr)] = expr.getGroupbyExprPairs(dsPair) +} + +case class WhenClauseExpr(expr: LogicalExpr) extends ClauseExpr { + def calculateOnly(values: Map[String, Any]): Option[Any] = expr.calculate(values) + val desc: String = s"WHEN ${expr.desc}" + val dataSources: Set[String] = expr.dataSources + override def getSubCacheExprs(ds: String): Iterable[Expr] = { + expr.getCacheExprs(ds) + } + override def getSubFinalCacheExprs(ds: String): Iterable[Expr] = { + expr.getFinalCacheExprs(ds) + } + override def getSubPersistExprs(ds: String): Iterable[Expr] = { + expr.getPersistExprs(ds) + } + + override def getGroupbyExprPairs(dsPair: (String, String)): Seq[(Expr, Expr)] = expr.getGroupbyExprPairs(dsPair) +} + +case class StatementExpr(whereClause: WhereClauseExpr, whenClauseOpt: Option[WhenClauseExpr]) extends ClauseExpr { + def calculateOnly(values: Map[String, Any]): Option[Any] = whereClause.calculate(values) + val desc: String = { + whenClauseOpt match { + case Some(expr) => s"${whereClause.desc} ${expr.desc}" + case _ => whereClause.desc + } + } + val dataSources: Set[String] = whereClause.dataSources + override def getSubCacheExprs(ds: String): Iterable[Expr] = { + whereClause.getCacheExprs(ds) + } + override def getSubFinalCacheExprs(ds: String): Iterable[Expr] = { + whereClause.getFinalCacheExprs(ds) + } + override def getSubPersistExprs(ds: String): Iterable[Expr] = { + whereClause.getPersistExprs(ds) + } + + override def getGroupbyExprPairs(dsPair: (String, String)): Seq[(Expr, Expr)] = whereClause.getGroupbyExprPairs(dsPair) +} + +//case class WhenClauseStatementExpr(expr: LogicalExpr, whenExpr: LogicalExpr) extends ClauseExpr { +// def calculateOnly(values: Map[String, Any]): Option[Any] = expr.calculate(values) +// val desc: String = s"${expr.desc} when ${whenExpr.desc}" +// +// override def valid(values: Map[String, Any]): Boolean = { +// whenExpr.calculate(values) match { +// case Some(r: Boolean) => r +// case _ => false +// } +// } +// +// val dataSources: Set[String] = expr.dataSources ++ whenExpr.dataSources +// override def getSubCacheExprs(ds: String): Iterable[Expr] = { +// expr.getCacheExprs(ds) ++ whenExpr.getCacheExprs(ds) +// } +// override def getSubFinalCacheExprs(ds: String): Iterable[Expr] = { +// expr.getFinalCacheExprs(ds) ++ whenExpr.getFinalCacheExprs(ds) +// } +// override def getSubPersistExprs(ds: String): Iterable[Expr] = { +// expr.getPersistExprs(ds) ++ whenExpr.getPersistExprs(ds) +// } +// +// override def getGroupbyExprPairs(dsPair: (String, String)): Seq[(Expr, Expr)] = { +// expr.getGroupbyExprPairs(dsPair) ++ whenExpr.getGroupbyExprPairs(dsPair) +// } +// override def getWhenClauseExpr(): Option[LogicalExpr] = Some(whenExpr) +//} \ No newline at end of file diff --git a/measure/src/main/scala/org/apache/griffin/measure/rule/expr/StatementExpr.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/expr/StatementExpr.scala deleted file mode 100644 index 7c4d18df0..000000000 --- a/measure/src/main/scala/org/apache/griffin/measure/rule/expr/StatementExpr.scala +++ /dev/null @@ -1,66 +0,0 @@ -/*- - * Licensed under the Apache License, Version 2.0 (the "License"); -you may not use this file except in compliance with the License. -You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, software -distributed under the License is distributed on an "AS IS" BASIS, -WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -See the License for the specific language governing permissions and -limitations under the License. - - */ -package org.apache.griffin.measure.rule.expr - - -trait StatementExpr extends Expr with AnalyzableExpr { - def valid(values: Map[String, Any]): Boolean = true - override def cacheUnit: Boolean = true -} - -case class SimpleStatementExpr(expr: LogicalExpr) extends StatementExpr { - def calculateOnly(values: Map[String, Any]): Option[Any] = expr.calculate(values) - val desc: String = expr.desc - val dataSources: Set[String] = expr.dataSources - override def getSubCacheExprs(ds: String): Iterable[Expr] = { - expr.getCacheExprs(ds) - } - override def getSubFinalCacheExprs(ds: String): Iterable[Expr] = { - expr.getFinalCacheExprs(ds) - } - override def getSubPersistExprs(ds: String): Iterable[Expr] = { - expr.getPersistExprs(ds) - } - - override def getGroupbyExprPairs(dsPair: (String, String)): Seq[(Expr, Expr)] = expr.getGroupbyExprPairs(dsPair) -} - -case class WhenClauseStatementExpr(expr: LogicalExpr, whenExpr: LogicalExpr) extends StatementExpr { - def calculateOnly(values: Map[String, Any]): Option[Any] = expr.calculate(values) - val desc: String = s"${expr.desc} when ${whenExpr.desc}" - - override def valid(values: Map[String, Any]): Boolean = { - whenExpr.calculate(values) match { - case Some(r: Boolean) => r - case _ => false - } - } - - val dataSources: Set[String] = expr.dataSources ++ whenExpr.dataSources - override def getSubCacheExprs(ds: String): Iterable[Expr] = { - expr.getCacheExprs(ds) ++ whenExpr.getCacheExprs(ds) - } - override def getSubFinalCacheExprs(ds: String): Iterable[Expr] = { - expr.getFinalCacheExprs(ds) ++ whenExpr.getFinalCacheExprs(ds) - } - override def getSubPersistExprs(ds: String): Iterable[Expr] = { - expr.getPersistExprs(ds) ++ whenExpr.getPersistExprs(ds) - } - - override def getGroupbyExprPairs(dsPair: (String, String)): Seq[(Expr, Expr)] = { - expr.getGroupbyExprPairs(dsPair) ++ whenExpr.getGroupbyExprPairs(dsPair) - } - override def getWhenClauseExpr(): Option[LogicalExpr] = Some(whenExpr) -} \ No newline at end of file diff --git a/measure/src/test/scala/org/apache/griffin/measure/algo/BatchAccuracyAlgoTest.scala b/measure/src/test/scala/org/apache/griffin/measure/algo/BatchAccuracyAlgoTest.scala index a62826115..222164548 100644 --- a/measure/src/test/scala/org/apache/griffin/measure/algo/BatchAccuracyAlgoTest.scala +++ b/measure/src/test/scala/org/apache/griffin/measure/algo/BatchAccuracyAlgoTest.scala @@ -1,188 +1,193 @@ -///*- -// * Licensed under the Apache License, Version 2.0 (the "License"); -//you may not use this file except in compliance with the License. -//You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -//Unless required by applicable law or agreed to in writing, software -//distributed under the License is distributed on an "AS IS" BASIS, -//WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -//See the License for the specific language governing permissions and -//limitations under the License. -// -// */ -// package org.apache.griffin.measure.algo -// -//import java.util.Date -// -//import org.apache.griffin.measure.config.params._ -//import org.apache.griffin.measure.config.params.env._ -//import org.apache.griffin.measure.config.params.user._ -//import org.apache.griffin.measure.config.reader._ -//import org.apache.griffin.measure.config.validator._ -//import org.apache.griffin.measure.connector.{DataConnector, DataConnectorFactory} -//import org.apache.griffin.measure.log.Loggable -//import org.apache.griffin.measure.rule.expr._ -//import org.apache.griffin.measure.rule.{ExprValueUtil, RuleAnalyzer, RuleFactory} -//import org.apache.spark.rdd.RDD -//import org.apache.spark.sql.SQLContext -//import org.apache.spark.{SparkConf, SparkContext} -//import org.junit.runner.RunWith -//import org.scalatest.junit.JUnitRunner -//import org.scalatest.{BeforeAndAfter, FunSuite, Matchers} -// -//import scala.util.{Failure, Success, Try} -// -// -//@RunWith(classOf[JUnitRunner]) -//class BatchAccuracyAlgoTest extends FunSuite with Matchers with BeforeAndAfter with Loggable { -// -// val envFile = "src/test/resources/env.json" -//// val confFile = "src/test/resources/config.json" -// val confFile = "{\"name\":\"accu1\",\"type\":\"accuracy\",\"source\":{\"type\":\"avro\",\"version\":\"1.7\",\"config\":{\"file.name\":\"src/test/resources/users_info_src.avro\"}},\"target\":{\"type\":\"avro\",\"version\":\"1.7\",\"config\":{\"file.name\":\"src/test/resources/users_info_target.avro\"}},\"evaluateRule\":{\"sampleRatio\":1,\"rules\":\"$source.user_id + 5 = $target.user_id + (2 + 3) AND $source.first_name + 12 = $target.first_name + (10 + 2) 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 AND (15 OR true) WHEN true AND $source.user_id > 10020\"}}" -// val envFsType = "local" -// val userFsType = "raw" -// -// val args = Array(envFile, confFile) -// -// var sc: SparkContext = _ -// var sqlContext: SQLContext = _ -// -// var allParam: AllParam = _ -// -// before { -// // read param files -// val envParam = readParamFile[EnvParam](envFile, envFsType) match { -// case Success(p) => p -// case Failure(ex) => { -// error(ex.getMessage) -// sys.exit(-2) -// } -// } -// val userParam = readParamFile[UserParam](confFile, userFsType) match { -// case Success(p) => p -// case Failure(ex) => { -// error(ex.getMessage) -// sys.exit(-2) -// } -// } -// allParam = AllParam(envParam, userParam) -// -// // validate param files -// validateParams(allParam) match { -// case Failure(ex) => { -// error(ex.getMessage) -// sys.exit(-3) -// } -// case _ => { -// info("params validation pass") -// } -// } -// -// val metricName = userParam.name -// val conf = new SparkConf().setMaster("local[*]").setAppName(metricName) -// sc = new SparkContext(conf) -// sqlContext = new SQLContext(sc) -// } -// -// test("algorithm") { -// Try { -// val envParam = allParam.envParam -// val userParam = allParam.userParam -// -// // start time -// val startTime = new Date().getTime() -// -// // get spark application id -// val applicationId = sc.applicationId -// -// // rules -// val ruleFactory = RuleFactory(userParam.evaluateRuleParam) -// val rule: StatementExpr = ruleFactory.generateRule() -// val ruleAnalyzer: RuleAnalyzer = RuleAnalyzer(rule) -// -// ruleAnalyzer.constCacheExprs.foreach(println) -// ruleAnalyzer.constFinalCacheExprs.foreach(println) -// -// // global cache data -// val constExprValueMap = ExprValueUtil.genExprValueMap(None, ruleAnalyzer.constCacheExprs, Map[String, Any]()) -// val finalConstExprValueMap = ExprValueUtil.updateExprValueMap(ruleAnalyzer.constFinalCacheExprs, constExprValueMap) -// -// // data connector -// val sourceDataConnector: DataConnector = -// DataConnectorFactory.getDataConnector(sqlContext, userParam.sourceParam, -// ruleAnalyzer.sourceRuleExprs, finalConstExprValueMap -// ) match { -// case Success(cntr) => { -// if (cntr.available) cntr -// else throw new Exception("source data not available!") -// } -// case Failure(ex) => throw ex -// } -// val targetDataConnector: DataConnector = -// DataConnectorFactory.getDataConnector(sqlContext, userParam.targetParam, -// ruleAnalyzer.targetRuleExprs, finalConstExprValueMap -// ) match { -// case Success(cntr) => { -// if (cntr.available) cntr -// else throw new Exception("target data not available!") -// } -// case Failure(ex) => throw ex -// } -// -// // get metadata -//// val sourceMetaData: Iterable[(String, String)] = sourceDataConnector.metaData() match { -//// case Success(md) => md -//// case Failure(ex) => throw ex -//// } -//// val targetMetaData: Iterable[(String, String)] = targetDataConnector.metaData() match { -//// case Success(md) => md -//// case Failure(ex) => throw ex -//// } -// -// // get data -// val sourceData: RDD[(Product, Map[String, Any])] = sourceDataConnector.data() match { -// case Success(dt) => dt +/*- + * Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. + + */ + package org.apache.griffin.measure.algo + +import java.util.Date + +import org.apache.griffin.measure.algo.batch.BatchAccuracyAlgo +import org.apache.griffin.measure.config.params._ +import org.apache.griffin.measure.config.params.env._ +import org.apache.griffin.measure.config.params.user._ +import org.apache.griffin.measure.config.reader._ +import org.apache.griffin.measure.config.validator._ +import org.apache.griffin.measure.connector.{BatchDataConnector, DataConnector, DataConnectorFactory} +import org.apache.griffin.measure.log.Loggable +import org.apache.griffin.measure.rule.expr._ +import org.apache.griffin.measure.rule.{ExprValueUtil, RuleAnalyzer, RuleFactory} +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.SQLContext +import org.apache.spark.{SparkConf, SparkContext} +import org.junit.runner.RunWith +import org.scalatest.junit.JUnitRunner +import org.scalatest.{BeforeAndAfter, FunSuite, Matchers} + +import scala.util.{Failure, Success, Try} + + +@RunWith(classOf[JUnitRunner]) +class BatchAccuracyAlgoTest extends FunSuite with Matchers with BeforeAndAfter with Loggable { + + val envFile = "src/test/resources/env.json" +// val confFile = "src/test/resources/config.json" + val confFile = "{\"name\":\"accu1\",\"type\":\"accuracy\",\"source\":{\"type\":\"avro\",\"version\":\"1.7\",\"config\":{\"file.name\":\"src/test/resources/users_info_src.avro\"}},\"target\":{\"type\":\"avro\",\"version\":\"1.7\",\"config\":{\"file.name\":\"src/test/resources/users_info_target.avro\"}},\"evaluateRule\":{\"sampleRatio\":1,\"rules\":\"$source.user_id + 5 = $target.user_id + (2 + 3) AND $source.first_name + 12 = $target.first_name + (10 + 2) 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 AND (15 OR true) WHEN true AND $source.user_id > 10020\"}}" + val envFsType = "local" + val userFsType = "raw" + + val args = Array(envFile, confFile) + + var sc: SparkContext = _ + var sqlContext: SQLContext = _ + + var allParam: AllParam = _ + + before { + // read param files + val envParam = readParamFile[EnvParam](envFile, envFsType) match { + case Success(p) => p + case Failure(ex) => { + error(ex.getMessage) + sys.exit(-2) + } + } + val userParam = readParamFile[UserParam](confFile, userFsType) match { + case Success(p) => p + case Failure(ex) => { + error(ex.getMessage) + sys.exit(-2) + } + } + allParam = AllParam(envParam, userParam) + + // validate param files + validateParams(allParam) match { + case Failure(ex) => { + error(ex.getMessage) + sys.exit(-3) + } + case _ => { + info("params validation pass") + } + } + + val metricName = userParam.name + val conf = new SparkConf().setMaster("local[*]").setAppName(metricName) + sc = new SparkContext(conf) + sqlContext = new SQLContext(sc) + } + + test("algorithm") { + Try { + val envParam = allParam.envParam + val userParam = allParam.userParam + + // start time + val startTime = new Date().getTime() + + // get spark application id + val applicationId = sc.applicationId + + // rules + val ruleFactory = RuleFactory(userParam.evaluateRuleParam) + val rule: StatementExpr = ruleFactory.generateRule() + val ruleAnalyzer: RuleAnalyzer = RuleAnalyzer(rule) + + ruleAnalyzer.constCacheExprs.foreach(println) + ruleAnalyzer.constFinalCacheExprs.foreach(println) + + // global cache data + val constExprValueMap = ExprValueUtil.genExprValueMaps(None, ruleAnalyzer.constCacheExprs, Map[String, Any]()) + val finalConstExprValueMap = ExprValueUtil.updateExprValueMaps(ruleAnalyzer.constFinalCacheExprs, constExprValueMap) + val finalConstMap = finalConstExprValueMap.headOption match { + case Some(m) => m + case _ => Map[String, Any]() + } + + // data connector + val sourceDataConnector: BatchDataConnector = + DataConnectorFactory.getBatchDataConnector(sqlContext, userParam.sourceParam, + ruleAnalyzer.sourceRuleExprs, finalConstMap + ) match { + case Success(cntr) => { + if (cntr.available) cntr + else throw new Exception("source data not available!") + } + case Failure(ex) => throw ex + } + val targetDataConnector: BatchDataConnector = + DataConnectorFactory.getBatchDataConnector(sqlContext, userParam.targetParam, + ruleAnalyzer.targetRuleExprs, finalConstMap + ) match { + case Success(cntr) => { + if (cntr.available) cntr + else throw new Exception("target data not available!") + } + case Failure(ex) => throw ex + } + + // get metadata +// val sourceMetaData: Iterable[(String, String)] = sourceDataConnector.metaData() match { +// case Success(md) => md // case Failure(ex) => throw ex // } -// val targetData: RDD[(Product, Map[String, Any])] = targetDataConnector.data() match { -// case Success(dt) => dt +// val targetMetaData: Iterable[(String, String)] = targetDataConnector.metaData() match { +// case Success(md) => md // case Failure(ex) => throw ex // } -// -// // my algo -// val algo = BatchAccuracyAlgo(allParam) -// -// // accuracy algorithm -// val (accuResult, missingRdd, matchedRdd) = algo.accuracy(sourceData, targetData, ruleAnalyzer) -// -// println(s"match percentage: ${accuResult.matchPercentage}, total count: ${accuResult.total}") -// -// missingRdd.map(rec => algo.record2String(rec, ruleAnalyzer.sourceRuleExprs.persistExprs, ruleAnalyzer.targetRuleExprs.persistExprs)).foreach(println) -// -// // end time -// val endTime = new Date().getTime -// println(s"using time: ${endTime - startTime} ms") -// } match { -// case Failure(ex) => { -// error(ex.getMessage) -// sys.exit(-4) -// } -// case _ => { -// info("calculation finished") -// } -// } -// } -// -// private def readParamFile[T <: Param](file: String, fsType: String)(implicit m : Manifest[T]): Try[T] = { -// val paramReader = ParamReaderFactory.getParamReader(file, fsType) -// paramReader.readConfig[T] -// } -// -// private def validateParams(allParam: AllParam): Try[Boolean] = { -// val allParamValidator = AllParamValidator() -// allParamValidator.validate(allParam) -// } -// -//} + + // get data + val sourceData: RDD[(Product, (Map[String, Any], Map[String, Any]))] = sourceDataConnector.data() match { + case Success(dt) => dt + case Failure(ex) => throw ex + } + val targetData: RDD[(Product, (Map[String, Any], Map[String, Any]))] = targetDataConnector.data() match { + case Success(dt) => dt + case Failure(ex) => throw ex + } + + // my algo + val algo = BatchAccuracyAlgo(allParam) + + // accuracy algorithm + val (accuResult, missingRdd, matchedRdd) = algo.accuracy(sourceData, targetData, ruleAnalyzer) + + println(s"match percentage: ${accuResult.matchPercentage}, total count: ${accuResult.total}") + + missingRdd.map(rec => algo.record2String(rec, ruleAnalyzer.sourceRuleExprs.persistExprs, ruleAnalyzer.targetRuleExprs.persistExprs)).foreach(println) + + // end time + val endTime = new Date().getTime + println(s"using time: ${endTime - startTime} ms") + } match { + case Failure(ex) => { + error(ex.getMessage) + sys.exit(-4) + } + case _ => { + info("calculation finished") + } + } + } + + private def readParamFile[T <: Param](file: String, fsType: String)(implicit m : Manifest[T]): Try[T] = { + val paramReader = ParamReaderFactory.getParamReader(file, fsType) + paramReader.readConfig[T] + } + + private def validateParams(allParam: AllParam): Try[Boolean] = { + val allParamValidator = AllParamValidator() + allParamValidator.validate(allParam) + } + +} diff --git a/measure/src/test/scala/org/apache/griffin/measure/cache/InfoCacheInstanceTest.scala b/measure/src/test/scala/org/apache/griffin/measure/cache/InfoCacheInstanceTest.scala new file mode 100644 index 000000000..bd99fb141 --- /dev/null +++ b/measure/src/test/scala/org/apache/griffin/measure/cache/InfoCacheInstanceTest.scala @@ -0,0 +1,73 @@ +/*- + * Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. + + */ +package org.apache.griffin.measure.cache + +import java.util.Date +import java.util.concurrent.{Executors, ThreadPoolExecutor, TimeUnit} + +import org.apache.curator.framework.recipes.locks.InterProcessMutex +import org.apache.curator.framework.{CuratorFramework, CuratorFrameworkFactory} +import org.apache.curator.retry.ExponentialBackoffRetry +import org.apache.griffin.measure.config.params.env.InfoCacheParam +import org.junit.runner.RunWith +import org.scalatest.junit.JUnitRunner +import org.scalatest.{BeforeAndAfter, FunSuite, Matchers} + +import scala.util.{Failure, Try} + +@RunWith(classOf[JUnitRunner]) +class InfoCacheInstanceTest extends FunSuite with Matchers with BeforeAndAfter { + + val map = Map[String, Any]( + ("hosts" -> "localhost:2181"), + ("namespace" -> "griffin/infocache"), + ("lock.path" -> "lock"), + ("mode" -> "persist"), + ("init.clear" -> true), + ("close.clear" -> false) + ) + val name = "ttt" + + val icp = InfoCacheParam("zk", map) + val icps = icp :: Nil + + before { + InfoCacheInstance.initInstance(icps, name) + InfoCacheInstance.init + } + + test ("others") { + InfoCacheInstance.available should be (true) + + val keys = List[String]( + "key1", "key2" + ) + val info = Map[String, String]( + ("key1" -> "value1"), + ("key2" -> "value2") + ) + + InfoCacheInstance.cacheInfo(info) should be (true) + InfoCacheInstance.readInfo(keys) should be (info) + InfoCacheInstance.deleteInfo(keys) +// InfoCacheInstance.readInfo(keys) should be (Map[String, String]()) + + } + + after { + InfoCacheInstance.close() + } + +} diff --git a/measure/src/test/scala/org/apache/griffin/measure/rule/RuleAnalyzerTest.scala b/measure/src/test/scala/org/apache/griffin/measure/rule/RuleAnalyzerTest.scala index 07a435837..cb0438034 100644 --- a/measure/src/test/scala/org/apache/griffin/measure/rule/RuleAnalyzerTest.scala +++ b/measure/src/test/scala/org/apache/griffin/measure/rule/RuleAnalyzerTest.scala @@ -41,8 +41,6 @@ class RuleAnalyzerTest extends FunSuite with BeforeAndAfter with Matchers { "$source['name']", "$source['age']", "$source['born']", "$source['born'] > 6 - 2 * 2")) ruleAnalyzer.sourceRuleExprs.persistExprs.map(_.desc) should be (List[String]( "$source['name']", "$source['age']", "$source['born']")) - ruleAnalyzer.sourceRuleExprs.whenClauseExprOpt.map(_.desc) should be (Some( - "$source['born'] > 6 - 2 * 2")) ruleAnalyzer.targetRuleExprs.groupbyExprs.map(_.desc) should be (List[String]( "$target['name']", "$target['age'] + 2 * 5")) @@ -52,8 +50,6 @@ class RuleAnalyzerTest extends FunSuite with BeforeAndAfter with Matchers { "$target['name']", "$target['age']", "$target['age'] + 2 * 5")) ruleAnalyzer.targetRuleExprs.persistExprs.map(_.desc) should be (List[String]( "$target['name']", "$target['age']")) - ruleAnalyzer.targetRuleExprs.whenClauseExprOpt.map(_.desc) should be (Some( - "$source['born'] > 6 - 2 * 2")) } From 40bfebda55da162c0f03c162a6e6e2caf68be870 Mon Sep 17 00:00:00 2001 From: Lionel Liu Date: Fri, 7 Jul 2017 15:36:14 +0800 Subject: [PATCH 012/111] t3 --- measure/src/test/resources/config.json | 2 +- .../apache/griffin/measure/algo/BatchAccuracyAlgoTest.scala | 6 +++--- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/measure/src/test/resources/config.json b/measure/src/test/resources/config.json index 65e0ed97d..4e3e96526 100644 --- a/measure/src/test/resources/config.json +++ b/measure/src/test/resources/config.json @@ -20,6 +20,6 @@ "evaluateRule": { "sampleRatio": 1, - "rules": "$source.user_id > 10020 AND $source.user_id + 5 = $target.user_id + (2 + 3) AND $source.first_name + 12 = $target.first_name + (10 + 2) 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" + "rules": "$source.user_id + 5 = $target.user_id + (2 + 3) AND $source.first_name + 12 = $target.first_name + (10 + 2) 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 WHEN $source.user_id > 10015" } } \ No newline at end of file diff --git a/measure/src/test/scala/org/apache/griffin/measure/algo/BatchAccuracyAlgoTest.scala b/measure/src/test/scala/org/apache/griffin/measure/algo/BatchAccuracyAlgoTest.scala index 222164548..57a908e20 100644 --- a/measure/src/test/scala/org/apache/griffin/measure/algo/BatchAccuracyAlgoTest.scala +++ b/measure/src/test/scala/org/apache/griffin/measure/algo/BatchAccuracyAlgoTest.scala @@ -40,10 +40,10 @@ import scala.util.{Failure, Success, Try} class BatchAccuracyAlgoTest extends FunSuite with Matchers with BeforeAndAfter with Loggable { val envFile = "src/test/resources/env.json" -// val confFile = "src/test/resources/config.json" - val confFile = "{\"name\":\"accu1\",\"type\":\"accuracy\",\"source\":{\"type\":\"avro\",\"version\":\"1.7\",\"config\":{\"file.name\":\"src/test/resources/users_info_src.avro\"}},\"target\":{\"type\":\"avro\",\"version\":\"1.7\",\"config\":{\"file.name\":\"src/test/resources/users_info_target.avro\"}},\"evaluateRule\":{\"sampleRatio\":1,\"rules\":\"$source.user_id + 5 = $target.user_id + (2 + 3) AND $source.first_name + 12 = $target.first_name + (10 + 2) 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 AND (15 OR true) WHEN true AND $source.user_id > 10020\"}}" + val confFile = "src/test/resources/config.json" +// val confFile = "{\"name\":\"accu1\",\"type\":\"accuracy\",\"source\":{\"type\":\"avro\",\"version\":\"1.7\",\"config\":{\"file.name\":\"src/test/resources/users_info_src.avro\"}},\"target\":{\"type\":\"avro\",\"version\":\"1.7\",\"config\":{\"file.name\":\"src/test/resources/users_info_target.avro\"}},\"evaluateRule\":{\"sampleRatio\":1,\"rules\":\"$source.user_id + 5 = $target.user_id + (2 + 3) AND $source.first_name + 12 = $target.first_name + (10 + 2) 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 AND (15 OR true) WHEN true AND $source.user_id > 10020\"}}" val envFsType = "local" - val userFsType = "raw" + val userFsType = "local" val args = Array(envFile, confFile) From 406502664c4ba33825f6da907a8b7978e248c7e0 Mon Sep 17 00:00:00 2001 From: Lionel Liu Date: Fri, 7 Jul 2017 17:18:55 +0800 Subject: [PATCH 013/111] version --- .../apache/griffin/measure/Application.scala | 32 ++- .../griffin/measure/algo/AccuracyAlgo.scala | 28 +- .../apache/griffin/measure/algo/Algo.scala | 32 ++- .../griffin/measure/algo/MeasureType.scala | 28 +- .../griffin/measure/algo/ProfileAlgo.scala | 28 +- .../algo/batch/BatchAccuracyAlgo.scala | 32 ++- .../measure/algo/batch/BatchProfileAlgo.scala | 32 ++- .../measure/algo/core/AccuracyCore.scala | 32 ++- .../measure/algo/core/ProfileCore.scala | 32 ++- .../streaming/StreamingAccuracyAlgo.scala | 32 ++- .../algo/streaming/StreamingProcess.scala | 32 ++- .../griffin/measure/cache/CacheLock.scala | 32 ++- .../griffin/measure/cache/InfoCache.scala | 32 ++- .../measure/cache/InfoCacheFactory.scala | 18 ++ .../measure/cache/InfoCacheInstance.scala | 32 ++- .../measure/cache/MultiCacheLock.scala | 32 ++- .../griffin/measure/cache/ZKCacheLock.scala | 32 ++- .../griffin/measure/cache/ZKInfoCache.scala | 32 ++- .../measure/config/params/AllParam.scala | 28 +- .../griffin/measure/config/params/Param.scala | 28 +- .../config/params/env/CleanerParam.scala | 28 +- .../measure/config/params/env/EnvParam.scala | 28 +- .../config/params/env/InfoCacheParam.scala | 28 +- .../config/params/env/PersistParam.scala | 28 +- .../config/params/env/SparkParam.scala | 28 +- .../params/user/DataConnectorParam.scala | 28 +- .../params/user/EvaluateRuleParam.scala | 28 +- .../config/params/user/UserParam.scala | 28 +- .../connector/AvroBatchDataConnector.scala | 32 ++- .../connector/BatchDataConnector.scala | 32 ++- .../connector/CacheDataConnector.scala | 18 ++ .../measure/connector/DataConnector.scala | 32 ++- .../connector/DataConnectorFactory.scala | 28 +- .../connector/HiveBatchDataConnector.scala | 32 ++- .../connector/KafkaDataConnector.scala | 32 ++- .../KafkaStreamingDataConnector.scala | 32 ++- .../connector/StreamingDataConnector.scala | 32 ++- .../connector/TempCacheDataConnector.scala | 18 ++ .../apache/griffin/measure/log/Loggable.scala | 32 ++- .../griffin/measure/persist/HdfsPersist.scala | 32 ++- .../griffin/measure/persist/HttpPersist.scala | 32 ++- .../measure/persist/LoggerPersist.scala | 28 +- .../measure/persist/MultiPersists.scala | 32 ++- .../griffin/measure/persist/Persist.scala | 32 ++- .../measure/persist/PersistFactory.scala | 32 ++- .../measure/result/AccuracyResult.scala | 32 ++- .../griffin/measure/result/DataInfo.scala | 32 ++- .../measure/result/ProfileResult.scala | 32 ++- .../griffin/measure/result/Result.scala | 32 ++- .../measure/rule/CalculationUtil.scala | 32 ++- .../rule/DataTypeCalculationUtil.scala | 18 ++ .../griffin/measure/rule/ExprValueUtil.scala | 32 ++- .../griffin/measure/rule/RuleAnalyzer.scala | 28 +- .../griffin/measure/rule/RuleFactory.scala | 32 ++- .../griffin/measure/rule/RuleParser.scala | 32 ++- .../measure/rule/SchemaValueCombineUtil.scala | 18 ++ .../measure/rule/expr/AnalyzableExpr.scala | 28 +- .../griffin/measure/rule/expr/Cacheable.scala | 28 +- .../measure/rule/expr/Calculatable.scala | 28 +- .../measure/rule/expr/ClauseExpr.scala | 28 +- .../measure/rule/expr/DataSourceable.scala | 28 +- .../measure/rule/expr/Describable.scala | 28 +- .../griffin/measure/rule/expr/Expr.scala | 32 ++- .../measure/rule/expr/ExprDescOnly.scala | 32 ++- .../measure/rule/expr/ExprIdCounter.scala | 32 ++- .../measure/rule/expr/FieldDescOnly.scala | 32 ++- .../measure/rule/expr/LiteralExpr.scala | 28 +- .../measure/rule/expr/LogicalExpr.scala | 28 +- .../griffin/measure/rule/expr/MathExpr.scala | 28 +- .../measure/rule/expr/SelectExpr.scala | 28 +- .../rule/func/DefaultFunctionDefine.scala | 28 +- .../measure/rule/func/FunctionDefine.scala | 28 +- .../measure/rule/func/FunctionUtil.scala | 28 +- .../griffin/measure/utils/HdfsUtil.scala | 32 ++- .../griffin/measure/utils/HttpUtil.scala | 32 ++- .../griffin/measure/utils/JsonUtil.scala | 32 ++- .../griffin/measure/utils/TimeUtil.scala | 28 +- .../src/test/resources/config-streaming.json | 57 +++++ measure/src/test/resources/env.json | 24 +- .../{ => batch}/BatchAccuracyAlgoTest.scala | 34 +-- .../{ => batch}/BatchProfileAlgoTest.scala | 32 ++- .../measure/algo/core/AccuracyCoreTest.scala | 28 +- .../measure/algo/core/ProfileCoreTest.scala | 28 +- .../streaming/StreamingAccuracyAlgoTest.scala | 239 ++++++++++++++++++ .../measure/cache/InfoCacheInstanceTest.scala | 32 ++- .../measure/cache/ZKCacheLockTest.scala | 32 ++- .../measure/cache/ZKInfoCacheTest.scala | 32 ++- .../reader/ParamRawStringReaderTest.scala | 32 ++- .../validator/AllParamValidatorTest.scala | 32 ++- .../measure/connector/ConnectorTest.scala | 32 ++- .../measure/persist/HdfsPersistTest.scala | 32 ++- .../measure/persist/HttpPersistTest.scala | 32 ++- .../measure/result/AccuracyResultTest.scala | 32 ++- .../measure/result/ProfileResultTest.scala | 32 ++- .../measure/rule/RuleAnalyzerTest.scala | 32 ++- .../measure/rule/RuleFactoryTest.scala | 32 ++- .../griffin/measure/rule/RuleParserTest.scala | 32 ++- .../griffin/measure/utils/JsonUtilTest.scala | 32 ++- 98 files changed, 1964 insertions(+), 1200 deletions(-) create mode 100644 measure/src/test/resources/config-streaming.json rename measure/src/test/scala/org/apache/griffin/measure/algo/{ => batch}/BatchAccuracyAlgoTest.scala (89%) rename measure/src/test/scala/org/apache/griffin/measure/algo/{ => batch}/BatchProfileAlgoTest.scala (86%) create mode 100644 measure/src/test/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgoTest.scala diff --git a/measure/src/main/scala/org/apache/griffin/measure/Application.scala b/measure/src/main/scala/org/apache/griffin/measure/Application.scala index a882fcea5..6d47f3134 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/Application.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/Application.scala @@ -1,17 +1,21 @@ -/*- - * Licensed under the Apache License, Version 2.0 (the "License"); -you may not use this file except in compliance with the License. -You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, software -distributed under the License is distributed on an "AS IS" BASIS, -WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -See the License for the specific language governing permissions and -limitations under the License. - - */ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ package org.apache.griffin.measure import org.apache.griffin.measure.algo._ diff --git a/measure/src/main/scala/org/apache/griffin/measure/algo/AccuracyAlgo.scala b/measure/src/main/scala/org/apache/griffin/measure/algo/AccuracyAlgo.scala index 1c2c1e574..7e0a56331 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/algo/AccuracyAlgo.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/algo/AccuracyAlgo.scala @@ -1,17 +1,21 @@ -/*- - * Licensed under the Apache License, Version 2.0 (the "License"); -you may not use this file except in compliance with the License. -You may obtain a copy of the License at +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at - http://www.apache.org/licenses/LICENSE-2.0 + http://www.apache.org/licenses/LICENSE-2.0 -Unless required by applicable law or agreed to in writing, software -distributed under the License is distributed on an "AS IS" BASIS, -WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -See the License for the specific language governing permissions and -limitations under the License. - - */ +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ package org.apache.griffin.measure.algo diff --git a/measure/src/main/scala/org/apache/griffin/measure/algo/Algo.scala b/measure/src/main/scala/org/apache/griffin/measure/algo/Algo.scala index c68c63572..82b71f1f2 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/algo/Algo.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/algo/Algo.scala @@ -1,17 +1,21 @@ -/*- - * Licensed under the Apache License, Version 2.0 (the "License"); -you may not use this file except in compliance with the License. -You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, software -distributed under the License is distributed on an "AS IS" BASIS, -WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -See the License for the specific language governing permissions and -limitations under the License. - - */ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ package org.apache.griffin.measure.algo import org.apache.griffin.measure.config.params.env._ diff --git a/measure/src/main/scala/org/apache/griffin/measure/algo/MeasureType.scala b/measure/src/main/scala/org/apache/griffin/measure/algo/MeasureType.scala index e827633d8..6bfff983c 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/algo/MeasureType.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/algo/MeasureType.scala @@ -1,17 +1,21 @@ -/*- - * Licensed under the Apache License, Version 2.0 (the "License"); -you may not use this file except in compliance with the License. -You may obtain a copy of the License at +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at - http://www.apache.org/licenses/LICENSE-2.0 + http://www.apache.org/licenses/LICENSE-2.0 -Unless required by applicable law or agreed to in writing, software -distributed under the License is distributed on an "AS IS" BASIS, -WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -See the License for the specific language governing permissions and -limitations under the License. - - */ +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ package org.apache.griffin.measure.algo object MeasureType { diff --git a/measure/src/main/scala/org/apache/griffin/measure/algo/ProfileAlgo.scala b/measure/src/main/scala/org/apache/griffin/measure/algo/ProfileAlgo.scala index 9108949dd..6ffc87aad 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/algo/ProfileAlgo.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/algo/ProfileAlgo.scala @@ -1,17 +1,21 @@ -/*- - * Licensed under the Apache License, Version 2.0 (the "License"); -you may not use this file except in compliance with the License. -You may obtain a copy of the License at +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at - http://www.apache.org/licenses/LICENSE-2.0 + http://www.apache.org/licenses/LICENSE-2.0 -Unless required by applicable law or agreed to in writing, software -distributed under the License is distributed on an "AS IS" BASIS, -WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -See the License for the specific language governing permissions and -limitations under the License. - - */ +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ package org.apache.griffin.measure.algo trait ProfileAlgo extends Algo { diff --git a/measure/src/main/scala/org/apache/griffin/measure/algo/batch/BatchAccuracyAlgo.scala b/measure/src/main/scala/org/apache/griffin/measure/algo/batch/BatchAccuracyAlgo.scala index 7e2e5ae56..7cc709aeb 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/algo/batch/BatchAccuracyAlgo.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/algo/batch/BatchAccuracyAlgo.scala @@ -1,17 +1,21 @@ -/*- - * Licensed under the Apache License, Version 2.0 (the "License"); -you may not use this file except in compliance with the License. -You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, software -distributed under the License is distributed on an "AS IS" BASIS, -WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -See the License for the specific language governing permissions and -limitations under the License. - - */ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ package org.apache.griffin.measure.algo.batch import java.util.Date diff --git a/measure/src/main/scala/org/apache/griffin/measure/algo/batch/BatchProfileAlgo.scala b/measure/src/main/scala/org/apache/griffin/measure/algo/batch/BatchProfileAlgo.scala index a5a89b19a..6b9cc564b 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/algo/batch/BatchProfileAlgo.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/algo/batch/BatchProfileAlgo.scala @@ -1,17 +1,21 @@ -/*- - * Licensed under the Apache License, Version 2.0 (the "License"); -you may not use this file except in compliance with the License. -You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, software -distributed under the License is distributed on an "AS IS" BASIS, -WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -See the License for the specific language governing permissions and -limitations under the License. - - */ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ package org.apache.griffin.measure.algo.batch import java.util.Date diff --git a/measure/src/main/scala/org/apache/griffin/measure/algo/core/AccuracyCore.scala b/measure/src/main/scala/org/apache/griffin/measure/algo/core/AccuracyCore.scala index d7d45d0a7..f3f827c0d 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/algo/core/AccuracyCore.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/algo/core/AccuracyCore.scala @@ -1,17 +1,21 @@ -/*- - * Licensed under the Apache License, Version 2.0 (the "License"); -you may not use this file except in compliance with the License. -You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, software -distributed under the License is distributed on an "AS IS" BASIS, -WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -See the License for the specific language governing permissions and -limitations under the License. - - */ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ package org.apache.griffin.measure.algo.core import org.apache.griffin.measure.rule.RuleAnalyzer diff --git a/measure/src/main/scala/org/apache/griffin/measure/algo/core/ProfileCore.scala b/measure/src/main/scala/org/apache/griffin/measure/algo/core/ProfileCore.scala index dc59e336c..2987f2fdf 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/algo/core/ProfileCore.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/algo/core/ProfileCore.scala @@ -1,17 +1,21 @@ -/*- - * Licensed under the Apache License, Version 2.0 (the "License"); -you may not use this file except in compliance with the License. -You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, software -distributed under the License is distributed on an "AS IS" BASIS, -WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -See the License for the specific language governing permissions and -limitations under the License. - - */ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ package org.apache.griffin.measure.algo.core import org.apache.griffin.measure.rule.RuleAnalyzer diff --git a/measure/src/main/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgo.scala b/measure/src/main/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgo.scala index cc87f25fe..5dd650073 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgo.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgo.scala @@ -1,17 +1,21 @@ -/*- - * Licensed under the Apache License, Version 2.0 (the "License"); -you may not use this file except in compliance with the License. -You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, software -distributed under the License is distributed on an "AS IS" BASIS, -WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -See the License for the specific language governing permissions and -limitations under the License. - - */ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ package org.apache.griffin.measure.algo.streaming import java.util.Date diff --git a/measure/src/main/scala/org/apache/griffin/measure/algo/streaming/StreamingProcess.scala b/measure/src/main/scala/org/apache/griffin/measure/algo/streaming/StreamingProcess.scala index 807795dcf..e5f2b4204 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/algo/streaming/StreamingProcess.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/algo/streaming/StreamingProcess.scala @@ -1,17 +1,21 @@ -/*- - * Licensed under the Apache License, Version 2.0 (the "License"); -you may not use this file except in compliance with the License. -You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, software -distributed under the License is distributed on an "AS IS" BASIS, -WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -See the License for the specific language governing permissions and -limitations under the License. - - */ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ package org.apache.griffin.measure.algo.streaming import java.util.concurrent.{Executors, ThreadPoolExecutor, TimeUnit} diff --git a/measure/src/main/scala/org/apache/griffin/measure/cache/CacheLock.scala b/measure/src/main/scala/org/apache/griffin/measure/cache/CacheLock.scala index 623be23e4..e1c7047ca 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/cache/CacheLock.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/cache/CacheLock.scala @@ -1,17 +1,21 @@ -/*- - * Licensed under the Apache License, Version 2.0 (the "License"); -you may not use this file except in compliance with the License. -You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, software -distributed under the License is distributed on an "AS IS" BASIS, -WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -See the License for the specific language governing permissions and -limitations under the License. - - */ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ package org.apache.griffin.measure.cache import java.util.concurrent.TimeUnit diff --git a/measure/src/main/scala/org/apache/griffin/measure/cache/InfoCache.scala b/measure/src/main/scala/org/apache/griffin/measure/cache/InfoCache.scala index a1ea09a74..3308fd6c2 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/cache/InfoCache.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/cache/InfoCache.scala @@ -1,17 +1,21 @@ -/*- - * Licensed under the Apache License, Version 2.0 (the "License"); -you may not use this file except in compliance with the License. -You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, software -distributed under the License is distributed on an "AS IS" BASIS, -WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -See the License for the specific language governing permissions and -limitations under the License. - - */ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ package org.apache.griffin.measure.cache import org.apache.griffin.measure.log.Loggable diff --git a/measure/src/main/scala/org/apache/griffin/measure/cache/InfoCacheFactory.scala b/measure/src/main/scala/org/apache/griffin/measure/cache/InfoCacheFactory.scala index 04d97241e..82369b2a3 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/cache/InfoCacheFactory.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/cache/InfoCacheFactory.scala @@ -1,3 +1,21 @@ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ package org.apache.griffin.measure.cache import org.apache.griffin.measure.config.params.env.InfoCacheParam diff --git a/measure/src/main/scala/org/apache/griffin/measure/cache/InfoCacheInstance.scala b/measure/src/main/scala/org/apache/griffin/measure/cache/InfoCacheInstance.scala index 779fb9a27..95c595ea2 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/cache/InfoCacheInstance.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/cache/InfoCacheInstance.scala @@ -1,17 +1,21 @@ -/*- - * Licensed under the Apache License, Version 2.0 (the "License"); -you may not use this file except in compliance with the License. -You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, software -distributed under the License is distributed on an "AS IS" BASIS, -WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -See the License for the specific language governing permissions and -limitations under the License. - - */ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ package org.apache.griffin.measure.cache import org.apache.griffin.measure.config.params.env.InfoCacheParam diff --git a/measure/src/main/scala/org/apache/griffin/measure/cache/MultiCacheLock.scala b/measure/src/main/scala/org/apache/griffin/measure/cache/MultiCacheLock.scala index 0a0a5859d..bc7d626a3 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/cache/MultiCacheLock.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/cache/MultiCacheLock.scala @@ -1,17 +1,21 @@ -/*- - * Licensed under the Apache License, Version 2.0 (the "License"); -you may not use this file except in compliance with the License. -You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, software -distributed under the License is distributed on an "AS IS" BASIS, -WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -See the License for the specific language governing permissions and -limitations under the License. - - */ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ package org.apache.griffin.measure.cache import java.util.concurrent.TimeUnit diff --git a/measure/src/main/scala/org/apache/griffin/measure/cache/ZKCacheLock.scala b/measure/src/main/scala/org/apache/griffin/measure/cache/ZKCacheLock.scala index d2d829404..0c24ac679 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/cache/ZKCacheLock.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/cache/ZKCacheLock.scala @@ -1,17 +1,21 @@ -/*- - * Licensed under the Apache License, Version 2.0 (the "License"); -you may not use this file except in compliance with the License. -You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, software -distributed under the License is distributed on an "AS IS" BASIS, -WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -See the License for the specific language governing permissions and -limitations under the License. - - */ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ package org.apache.griffin.measure.cache import java.util.concurrent.TimeUnit diff --git a/measure/src/main/scala/org/apache/griffin/measure/cache/ZKInfoCache.scala b/measure/src/main/scala/org/apache/griffin/measure/cache/ZKInfoCache.scala index 33d7acda9..72b0de280 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/cache/ZKInfoCache.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/cache/ZKInfoCache.scala @@ -1,17 +1,21 @@ -/*- - * Licensed under the Apache License, Version 2.0 (the "License"); -you may not use this file except in compliance with the License. -You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, software -distributed under the License is distributed on an "AS IS" BASIS, -WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -See the License for the specific language governing permissions and -limitations under the License. - - */ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ package org.apache.griffin.measure.cache import org.apache.curator.framework.imps.CuratorFrameworkState diff --git a/measure/src/main/scala/org/apache/griffin/measure/config/params/AllParam.scala b/measure/src/main/scala/org/apache/griffin/measure/config/params/AllParam.scala index 1e0f86a08..8990564be 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/config/params/AllParam.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/config/params/AllParam.scala @@ -1,17 +1,21 @@ -/*- - * Licensed under the Apache License, Version 2.0 (the "License"); -you may not use this file except in compliance with the License. -You may obtain a copy of the License at +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at - http://www.apache.org/licenses/LICENSE-2.0 + http://www.apache.org/licenses/LICENSE-2.0 -Unless required by applicable law or agreed to in writing, software -distributed under the License is distributed on an "AS IS" BASIS, -WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -See the License for the specific language governing permissions and -limitations under the License. - - */ +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ package org.apache.griffin.measure.config.params import com.fasterxml.jackson.annotation.{JsonInclude, JsonProperty} diff --git a/measure/src/main/scala/org/apache/griffin/measure/config/params/Param.scala b/measure/src/main/scala/org/apache/griffin/measure/config/params/Param.scala index 46ebdeb92..d3484a139 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/config/params/Param.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/config/params/Param.scala @@ -1,17 +1,21 @@ -/*- - * Licensed under the Apache License, Version 2.0 (the "License"); -you may not use this file except in compliance with the License. -You may obtain a copy of the License at +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at - http://www.apache.org/licenses/LICENSE-2.0 + http://www.apache.org/licenses/LICENSE-2.0 -Unless required by applicable law or agreed to in writing, software -distributed under the License is distributed on an "AS IS" BASIS, -WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -See the License for the specific language governing permissions and -limitations under the License. - - */ +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ package org.apache.griffin.measure.config.params trait Param extends Serializable { diff --git a/measure/src/main/scala/org/apache/griffin/measure/config/params/env/CleanerParam.scala b/measure/src/main/scala/org/apache/griffin/measure/config/params/env/CleanerParam.scala index e5d2f093b..6c1ff9494 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/config/params/env/CleanerParam.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/config/params/env/CleanerParam.scala @@ -1,17 +1,21 @@ -/*- - * Licensed under the Apache License, Version 2.0 (the "License"); -you may not use this file except in compliance with the License. -You may obtain a copy of the License at +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at - http://www.apache.org/licenses/LICENSE-2.0 + http://www.apache.org/licenses/LICENSE-2.0 -Unless required by applicable law or agreed to in writing, software -distributed under the License is distributed on an "AS IS" BASIS, -WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -See the License for the specific language governing permissions and -limitations under the License. - - */ +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ package org.apache.griffin.measure.config.params.env import com.fasterxml.jackson.annotation.{JsonInclude, JsonProperty} diff --git a/measure/src/main/scala/org/apache/griffin/measure/config/params/env/EnvParam.scala b/measure/src/main/scala/org/apache/griffin/measure/config/params/env/EnvParam.scala index cab779536..ad87a5f2c 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/config/params/env/EnvParam.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/config/params/env/EnvParam.scala @@ -1,17 +1,21 @@ -/*- - * Licensed under the Apache License, Version 2.0 (the "License"); -you may not use this file except in compliance with the License. -You may obtain a copy of the License at +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at - http://www.apache.org/licenses/LICENSE-2.0 + http://www.apache.org/licenses/LICENSE-2.0 -Unless required by applicable law or agreed to in writing, software -distributed under the License is distributed on an "AS IS" BASIS, -WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -See the License for the specific language governing permissions and -limitations under the License. - - */ +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ package org.apache.griffin.measure.config.params.env import com.fasterxml.jackson.annotation.{JsonInclude, JsonProperty} diff --git a/measure/src/main/scala/org/apache/griffin/measure/config/params/env/InfoCacheParam.scala b/measure/src/main/scala/org/apache/griffin/measure/config/params/env/InfoCacheParam.scala index 6ddade9d0..be588f9e8 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/config/params/env/InfoCacheParam.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/config/params/env/InfoCacheParam.scala @@ -1,17 +1,21 @@ -/*- - * Licensed under the Apache License, Version 2.0 (the "License"); -you may not use this file except in compliance with the License. -You may obtain a copy of the License at +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at - http://www.apache.org/licenses/LICENSE-2.0 + http://www.apache.org/licenses/LICENSE-2.0 -Unless required by applicable law or agreed to in writing, software -distributed under the License is distributed on an "AS IS" BASIS, -WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -See the License for the specific language governing permissions and -limitations under the License. - - */ +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ package org.apache.griffin.measure.config.params.env import com.fasterxml.jackson.annotation.{JsonInclude, JsonProperty} diff --git a/measure/src/main/scala/org/apache/griffin/measure/config/params/env/PersistParam.scala b/measure/src/main/scala/org/apache/griffin/measure/config/params/env/PersistParam.scala index 4aaa059d4..68b9bc899 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/config/params/env/PersistParam.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/config/params/env/PersistParam.scala @@ -1,17 +1,21 @@ -/*- - * Licensed under the Apache License, Version 2.0 (the "License"); -you may not use this file except in compliance with the License. -You may obtain a copy of the License at +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at - http://www.apache.org/licenses/LICENSE-2.0 + http://www.apache.org/licenses/LICENSE-2.0 -Unless required by applicable law or agreed to in writing, software -distributed under the License is distributed on an "AS IS" BASIS, -WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -See the License for the specific language governing permissions and -limitations under the License. - - */ +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ package org.apache.griffin.measure.config.params.env import com.fasterxml.jackson.annotation.{JsonInclude, JsonProperty} diff --git a/measure/src/main/scala/org/apache/griffin/measure/config/params/env/SparkParam.scala b/measure/src/main/scala/org/apache/griffin/measure/config/params/env/SparkParam.scala index 1927f81a9..872a870ec 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/config/params/env/SparkParam.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/config/params/env/SparkParam.scala @@ -1,17 +1,21 @@ -/*- - * Licensed under the Apache License, Version 2.0 (the "License"); -you may not use this file except in compliance with the License. -You may obtain a copy of the License at +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at - http://www.apache.org/licenses/LICENSE-2.0 + http://www.apache.org/licenses/LICENSE-2.0 -Unless required by applicable law or agreed to in writing, software -distributed under the License is distributed on an "AS IS" BASIS, -WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -See the License for the specific language governing permissions and -limitations under the License. - - */ +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ package org.apache.griffin.measure.config.params.env import com.fasterxml.jackson.annotation.{JsonInclude, JsonProperty} diff --git a/measure/src/main/scala/org/apache/griffin/measure/config/params/user/DataConnectorParam.scala b/measure/src/main/scala/org/apache/griffin/measure/config/params/user/DataConnectorParam.scala index 56aa2a1ff..233c15228 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/config/params/user/DataConnectorParam.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/config/params/user/DataConnectorParam.scala @@ -1,17 +1,21 @@ -/*- - * Licensed under the Apache License, Version 2.0 (the "License"); -you may not use this file except in compliance with the License. -You may obtain a copy of the License at +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at - http://www.apache.org/licenses/LICENSE-2.0 + http://www.apache.org/licenses/LICENSE-2.0 -Unless required by applicable law or agreed to in writing, software -distributed under the License is distributed on an "AS IS" BASIS, -WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -See the License for the specific language governing permissions and -limitations under the License. - - */ +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ package org.apache.griffin.measure.config.params.user import com.fasterxml.jackson.annotation.{JsonInclude, JsonProperty} diff --git a/measure/src/main/scala/org/apache/griffin/measure/config/params/user/EvaluateRuleParam.scala b/measure/src/main/scala/org/apache/griffin/measure/config/params/user/EvaluateRuleParam.scala index 8db6ec21c..6ee978344 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/config/params/user/EvaluateRuleParam.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/config/params/user/EvaluateRuleParam.scala @@ -1,17 +1,21 @@ -/*- - * Licensed under the Apache License, Version 2.0 (the "License"); -you may not use this file except in compliance with the License. -You may obtain a copy of the License at +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at - http://www.apache.org/licenses/LICENSE-2.0 + http://www.apache.org/licenses/LICENSE-2.0 -Unless required by applicable law or agreed to in writing, software -distributed under the License is distributed on an "AS IS" BASIS, -WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -See the License for the specific language governing permissions and -limitations under the License. - - */ +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ package org.apache.griffin.measure.config.params.user import com.fasterxml.jackson.annotation.{JsonInclude, JsonProperty} diff --git a/measure/src/main/scala/org/apache/griffin/measure/config/params/user/UserParam.scala b/measure/src/main/scala/org/apache/griffin/measure/config/params/user/UserParam.scala index 24bd25eb4..cd44e0b15 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/config/params/user/UserParam.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/config/params/user/UserParam.scala @@ -1,17 +1,21 @@ -/*- - * Licensed under the Apache License, Version 2.0 (the "License"); -you may not use this file except in compliance with the License. -You may obtain a copy of the License at +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at - http://www.apache.org/licenses/LICENSE-2.0 + http://www.apache.org/licenses/LICENSE-2.0 -Unless required by applicable law or agreed to in writing, software -distributed under the License is distributed on an "AS IS" BASIS, -WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -See the License for the specific language governing permissions and -limitations under the License. - - */ +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ package org.apache.griffin.measure.config.params.user import com.fasterxml.jackson.annotation.{JsonInclude, JsonProperty} diff --git a/measure/src/main/scala/org/apache/griffin/measure/connector/AvroBatchDataConnector.scala b/measure/src/main/scala/org/apache/griffin/measure/connector/AvroBatchDataConnector.scala index 2736d270b..b2f2f47a8 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/connector/AvroBatchDataConnector.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/connector/AvroBatchDataConnector.scala @@ -1,17 +1,21 @@ -/*- - * Licensed under the Apache License, Version 2.0 (the "License"); -you may not use this file except in compliance with the License. -You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, software -distributed under the License is distributed on an "AS IS" BASIS, -WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -See the License for the specific language governing permissions and -limitations under the License. - - */ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ package org.apache.griffin.measure.connector import org.apache.griffin.measure.rule.expr._ diff --git a/measure/src/main/scala/org/apache/griffin/measure/connector/BatchDataConnector.scala b/measure/src/main/scala/org/apache/griffin/measure/connector/BatchDataConnector.scala index 9e8b1e02d..656588c9c 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/connector/BatchDataConnector.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/connector/BatchDataConnector.scala @@ -1,17 +1,21 @@ -/*- - * Licensed under the Apache License, Version 2.0 (the "License"); -you may not use this file except in compliance with the License. -You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, software -distributed under the License is distributed on an "AS IS" BASIS, -WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -See the License for the specific language governing permissions and -limitations under the License. - - */ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ package org.apache.griffin.measure.connector import org.apache.spark.rdd.RDD diff --git a/measure/src/main/scala/org/apache/griffin/measure/connector/CacheDataConnector.scala b/measure/src/main/scala/org/apache/griffin/measure/connector/CacheDataConnector.scala index 2f8b8166c..bff9c76e8 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/connector/CacheDataConnector.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/connector/CacheDataConnector.scala @@ -1,3 +1,21 @@ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ package org.apache.griffin.measure.connector import org.apache.griffin.measure.cache.{InfoCacheInstance, ZKInfoCache} diff --git a/measure/src/main/scala/org/apache/griffin/measure/connector/DataConnector.scala b/measure/src/main/scala/org/apache/griffin/measure/connector/DataConnector.scala index 2ac32499f..3f76fe39b 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/connector/DataConnector.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/connector/DataConnector.scala @@ -1,17 +1,21 @@ -/*- - * Licensed under the Apache License, Version 2.0 (the "License"); -you may not use this file except in compliance with the License. -You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, software -distributed under the License is distributed on an "AS IS" BASIS, -WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -See the License for the specific language governing permissions and -limitations under the License. - - */ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ package org.apache.griffin.measure.connector import org.apache.griffin.measure.log.Loggable diff --git a/measure/src/main/scala/org/apache/griffin/measure/connector/DataConnectorFactory.scala b/measure/src/main/scala/org/apache/griffin/measure/connector/DataConnectorFactory.scala index 82fea04b0..962d4f6ad 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/connector/DataConnectorFactory.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/connector/DataConnectorFactory.scala @@ -1,17 +1,21 @@ -/*- - * Licensed under the Apache License, Version 2.0 (the "License"); -you may not use this file except in compliance with the License. -You may obtain a copy of the License at +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at - http://www.apache.org/licenses/LICENSE-2.0 + http://www.apache.org/licenses/LICENSE-2.0 -Unless required by applicable law or agreed to in writing, software -distributed under the License is distributed on an "AS IS" BASIS, -WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -See the License for the specific language governing permissions and -limitations under the License. - - */ +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ package org.apache.griffin.measure.connector import kafka.serializer.StringDecoder diff --git a/measure/src/main/scala/org/apache/griffin/measure/connector/HiveBatchDataConnector.scala b/measure/src/main/scala/org/apache/griffin/measure/connector/HiveBatchDataConnector.scala index 0318d9750..59b3e6df1 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/connector/HiveBatchDataConnector.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/connector/HiveBatchDataConnector.scala @@ -1,17 +1,21 @@ -/*- - * Licensed under the Apache License, Version 2.0 (the "License"); -you may not use this file except in compliance with the License. -You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, software -distributed under the License is distributed on an "AS IS" BASIS, -WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -See the License for the specific language governing permissions and -limitations under the License. - - */ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ package org.apache.griffin.measure.connector import org.apache.griffin.measure.result._ diff --git a/measure/src/main/scala/org/apache/griffin/measure/connector/KafkaDataConnector.scala b/measure/src/main/scala/org/apache/griffin/measure/connector/KafkaDataConnector.scala index 603a05bd3..1a882da19 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/connector/KafkaDataConnector.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/connector/KafkaDataConnector.scala @@ -1,17 +1,21 @@ -/*- - * Licensed under the Apache License, Version 2.0 (the "License"); -you may not use this file except in compliance with the License. -You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, software -distributed under the License is distributed on an "AS IS" BASIS, -WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -See the License for the specific language governing permissions and -limitations under the License. - - */ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ package org.apache.griffin.measure.connector import org.apache.griffin.measure.config.params.user.DataConnectorParam diff --git a/measure/src/main/scala/org/apache/griffin/measure/connector/KafkaStreamingDataConnector.scala b/measure/src/main/scala/org/apache/griffin/measure/connector/KafkaStreamingDataConnector.scala index 1db8dc513..8a02601c2 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/connector/KafkaStreamingDataConnector.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/connector/KafkaStreamingDataConnector.scala @@ -1,17 +1,21 @@ -/*- - * Licensed under the Apache License, Version 2.0 (the "License"); -you may not use this file except in compliance with the License. -You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, software -distributed under the License is distributed on an "AS IS" BASIS, -WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -See the License for the specific language governing permissions and -limitations under the License. - - */ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ package org.apache.griffin.measure.connector import kafka.serializer.Decoder diff --git a/measure/src/main/scala/org/apache/griffin/measure/connector/StreamingDataConnector.scala b/measure/src/main/scala/org/apache/griffin/measure/connector/StreamingDataConnector.scala index b0bc80e52..9e6330cbd 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/connector/StreamingDataConnector.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/connector/StreamingDataConnector.scala @@ -1,17 +1,21 @@ -/*- - * Licensed under the Apache License, Version 2.0 (the "License"); -you may not use this file except in compliance with the License. -You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, software -distributed under the License is distributed on an "AS IS" BASIS, -WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -See the License for the specific language governing permissions and -limitations under the License. - - */ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ package org.apache.griffin.measure.connector import org.apache.spark.streaming.dstream.InputDStream diff --git a/measure/src/main/scala/org/apache/griffin/measure/connector/TempCacheDataConnector.scala b/measure/src/main/scala/org/apache/griffin/measure/connector/TempCacheDataConnector.scala index 1dec51b27..879a1ad7d 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/connector/TempCacheDataConnector.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/connector/TempCacheDataConnector.scala @@ -1,3 +1,21 @@ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ package org.apache.griffin.measure.connector import org.apache.spark.sql.{DataFrame, SQLContext} diff --git a/measure/src/main/scala/org/apache/griffin/measure/log/Loggable.scala b/measure/src/main/scala/org/apache/griffin/measure/log/Loggable.scala index bd4f13b47..265a8cda2 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/log/Loggable.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/log/Loggable.scala @@ -1,17 +1,21 @@ -/*- - * Licensed under the Apache License, Version 2.0 (the "License"); -you may not use this file except in compliance with the License. -You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, software -distributed under the License is distributed on an "AS IS" BASIS, -WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -See the License for the specific language governing permissions and -limitations under the License. - - */ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ package org.apache.griffin.measure.log import org.slf4j.LoggerFactory diff --git a/measure/src/main/scala/org/apache/griffin/measure/persist/HdfsPersist.scala b/measure/src/main/scala/org/apache/griffin/measure/persist/HdfsPersist.scala index 67b86f135..68a5fb899 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/persist/HdfsPersist.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/persist/HdfsPersist.scala @@ -1,17 +1,21 @@ -/*- - * Licensed under the Apache License, Version 2.0 (the "License"); -you may not use this file except in compliance with the License. -You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, software -distributed under the License is distributed on an "AS IS" BASIS, -WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -See the License for the specific language governing permissions and -limitations under the License. - - */ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ package org.apache.griffin.measure.persist import java.util.Date diff --git a/measure/src/main/scala/org/apache/griffin/measure/persist/HttpPersist.scala b/measure/src/main/scala/org/apache/griffin/measure/persist/HttpPersist.scala index 8c0438613..0b2ed3adc 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/persist/HttpPersist.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/persist/HttpPersist.scala @@ -1,17 +1,21 @@ -/*- - * Licensed under the Apache License, Version 2.0 (the "License"); -you may not use this file except in compliance with the License. -You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, software -distributed under the License is distributed on an "AS IS" BASIS, -WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -See the License for the specific language governing permissions and -limitations under the License. - - */ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ package org.apache.griffin.measure.persist import org.apache.griffin.measure.result._ diff --git a/measure/src/main/scala/org/apache/griffin/measure/persist/LoggerPersist.scala b/measure/src/main/scala/org/apache/griffin/measure/persist/LoggerPersist.scala index db45f4faa..71c457fef 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/persist/LoggerPersist.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/persist/LoggerPersist.scala @@ -1,17 +1,21 @@ -/*- - * Licensed under the Apache License, Version 2.0 (the "License"); -you may not use this file except in compliance with the License. -You may obtain a copy of the License at +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at - http://www.apache.org/licenses/LICENSE-2.0 + http://www.apache.org/licenses/LICENSE-2.0 -Unless required by applicable law or agreed to in writing, software -distributed under the License is distributed on an "AS IS" BASIS, -WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -See the License for the specific language governing permissions and -limitations under the License. - - */ +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ package org.apache.griffin.measure.persist import java.util.Date diff --git a/measure/src/main/scala/org/apache/griffin/measure/persist/MultiPersists.scala b/measure/src/main/scala/org/apache/griffin/measure/persist/MultiPersists.scala index 5674729f9..de30ad062 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/persist/MultiPersists.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/persist/MultiPersists.scala @@ -1,17 +1,21 @@ -/*- - * Licensed under the Apache License, Version 2.0 (the "License"); -you may not use this file except in compliance with the License. -You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, software -distributed under the License is distributed on an "AS IS" BASIS, -WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -See the License for the specific language governing permissions and -limitations under the License. - - */ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ package org.apache.griffin.measure.persist import org.apache.griffin.measure.result._ diff --git a/measure/src/main/scala/org/apache/griffin/measure/persist/Persist.scala b/measure/src/main/scala/org/apache/griffin/measure/persist/Persist.scala index 0ecfa213f..01f205d97 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/persist/Persist.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/persist/Persist.scala @@ -1,17 +1,21 @@ -/*- - * Licensed under the Apache License, Version 2.0 (the "License"); -you may not use this file except in compliance with the License. -You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, software -distributed under the License is distributed on an "AS IS" BASIS, -WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -See the License for the specific language governing permissions and -limitations under the License. - - */ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ package org.apache.griffin.measure.persist import org.apache.griffin.measure.log.Loggable diff --git a/measure/src/main/scala/org/apache/griffin/measure/persist/PersistFactory.scala b/measure/src/main/scala/org/apache/griffin/measure/persist/PersistFactory.scala index 43bb2a83e..2532d3ea1 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/persist/PersistFactory.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/persist/PersistFactory.scala @@ -1,17 +1,21 @@ -/*- - * Licensed under the Apache License, Version 2.0 (the "License"); -you may not use this file except in compliance with the License. -You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, software -distributed under the License is distributed on an "AS IS" BASIS, -WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -See the License for the specific language governing permissions and -limitations under the License. - - */ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ package org.apache.griffin.measure.persist import org.apache.griffin.measure.config.params.env._ diff --git a/measure/src/main/scala/org/apache/griffin/measure/result/AccuracyResult.scala b/measure/src/main/scala/org/apache/griffin/measure/result/AccuracyResult.scala index 0ffab6670..16bb772c9 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/result/AccuracyResult.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/result/AccuracyResult.scala @@ -1,17 +1,21 @@ -/*- - * Licensed under the Apache License, Version 2.0 (the "License"); -you may not use this file except in compliance with the License. -You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, software -distributed under the License is distributed on an "AS IS" BASIS, -WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -See the License for the specific language governing permissions and -limitations under the License. - - */ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ package org.apache.griffin.measure.result // result for accuracy: miss count, total count diff --git a/measure/src/main/scala/org/apache/griffin/measure/result/DataInfo.scala b/measure/src/main/scala/org/apache/griffin/measure/result/DataInfo.scala index e2cf5e17c..7ec0783a7 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/result/DataInfo.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/result/DataInfo.scala @@ -1,17 +1,21 @@ -/*- - * Licensed under the Apache License, Version 2.0 (the "License"); -you may not use this file except in compliance with the License. -You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, software -distributed under the License is distributed on an "AS IS" BASIS, -WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -See the License for the specific language governing permissions and -limitations under the License. - - */ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ package org.apache.griffin.measure.result diff --git a/measure/src/main/scala/org/apache/griffin/measure/result/ProfileResult.scala b/measure/src/main/scala/org/apache/griffin/measure/result/ProfileResult.scala index 0415c1f99..803416ecf 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/result/ProfileResult.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/result/ProfileResult.scala @@ -1,17 +1,21 @@ -/*- - * Licensed under the Apache License, Version 2.0 (the "License"); -you may not use this file except in compliance with the License. -You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, software -distributed under the License is distributed on an "AS IS" BASIS, -WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -See the License for the specific language governing permissions and -limitations under the License. - - */ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ package org.apache.griffin.measure.result // result for profile: match count, total count diff --git a/measure/src/main/scala/org/apache/griffin/measure/result/Result.scala b/measure/src/main/scala/org/apache/griffin/measure/result/Result.scala index 5ac8935b7..6dcd9a1ff 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/result/Result.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/result/Result.scala @@ -1,17 +1,21 @@ -/*- - * Licensed under the Apache License, Version 2.0 (the "License"); -you may not use this file except in compliance with the License. -You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, software -distributed under the License is distributed on an "AS IS" BASIS, -WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -See the License for the specific language governing permissions and -limitations under the License. - - */ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ package org.apache.griffin.measure.result diff --git a/measure/src/main/scala/org/apache/griffin/measure/rule/CalculationUtil.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/CalculationUtil.scala index 797d3ebef..c96901280 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rule/CalculationUtil.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rule/CalculationUtil.scala @@ -1,17 +1,21 @@ -/*- - * Licensed under the Apache License, Version 2.0 (the "License"); -you may not use this file except in compliance with the License. -You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, software -distributed under the License is distributed on an "AS IS" BASIS, -WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -See the License for the specific language governing permissions and -limitations under the License. - - */ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ package org.apache.griffin.measure.rule import scala.util.{Success, Try} diff --git a/measure/src/main/scala/org/apache/griffin/measure/rule/DataTypeCalculationUtil.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/DataTypeCalculationUtil.scala index 1e865699e..0a3781d00 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rule/DataTypeCalculationUtil.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rule/DataTypeCalculationUtil.scala @@ -1,3 +1,21 @@ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ package org.apache.griffin.measure.rule import org.apache.spark.sql.types._ diff --git a/measure/src/main/scala/org/apache/griffin/measure/rule/ExprValueUtil.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/ExprValueUtil.scala index f5739f9e0..7d7459c0a 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rule/ExprValueUtil.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rule/ExprValueUtil.scala @@ -1,17 +1,21 @@ -/*- - * Licensed under the Apache License, Version 2.0 (the "License"); -you may not use this file except in compliance with the License. -You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, software -distributed under the License is distributed on an "AS IS" BASIS, -WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -See the License for the specific language governing permissions and -limitations under the License. - - */ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ package org.apache.griffin.measure.rule import org.apache.griffin.measure.rule.expr._ diff --git a/measure/src/main/scala/org/apache/griffin/measure/rule/RuleAnalyzer.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/RuleAnalyzer.scala index 00e059480..9e7b6dcde 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rule/RuleAnalyzer.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rule/RuleAnalyzer.scala @@ -1,17 +1,21 @@ -/*- - * Licensed under the Apache License, Version 2.0 (the "License"); -you may not use this file except in compliance with the License. -You may obtain a copy of the License at +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at - http://www.apache.org/licenses/LICENSE-2.0 + http://www.apache.org/licenses/LICENSE-2.0 -Unless required by applicable law or agreed to in writing, software -distributed under the License is distributed on an "AS IS" BASIS, -WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -See the License for the specific language governing permissions and -limitations under the License. - - */ +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ package org.apache.griffin.measure.rule import org.apache.griffin.measure.rule.expr._ diff --git a/measure/src/main/scala/org/apache/griffin/measure/rule/RuleFactory.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/RuleFactory.scala index 06d4a7b44..bbaf5cb29 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rule/RuleFactory.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rule/RuleFactory.scala @@ -1,17 +1,21 @@ -/*- - * Licensed under the Apache License, Version 2.0 (the "License"); -you may not use this file except in compliance with the License. -You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, software -distributed under the License is distributed on an "AS IS" BASIS, -WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -See the License for the specific language governing permissions and -limitations under the License. - - */ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ package org.apache.griffin.measure.rule import org.apache.griffin.measure.config.params.user._ diff --git a/measure/src/main/scala/org/apache/griffin/measure/rule/RuleParser.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/RuleParser.scala index cedc8350f..99831f2a1 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rule/RuleParser.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rule/RuleParser.scala @@ -1,17 +1,21 @@ -/*- - * Licensed under the Apache License, Version 2.0 (the "License"); -you may not use this file except in compliance with the License. -You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, software -distributed under the License is distributed on an "AS IS" BASIS, -WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -See the License for the specific language governing permissions and -limitations under the License. - - */ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ package org.apache.griffin.measure.rule import org.apache.griffin.measure.rule.expr._ diff --git a/measure/src/main/scala/org/apache/griffin/measure/rule/SchemaValueCombineUtil.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/SchemaValueCombineUtil.scala index 00a794c5a..ae12ba3a0 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rule/SchemaValueCombineUtil.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rule/SchemaValueCombineUtil.scala @@ -1,3 +1,21 @@ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ package org.apache.griffin.measure.rule object SchemaValueCombineUtil { diff --git a/measure/src/main/scala/org/apache/griffin/measure/rule/expr/AnalyzableExpr.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/expr/AnalyzableExpr.scala index a2dd834fc..aefcaaddf 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rule/expr/AnalyzableExpr.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rule/expr/AnalyzableExpr.scala @@ -1,17 +1,21 @@ -/*- - * Licensed under the Apache License, Version 2.0 (the "License"); -you may not use this file except in compliance with the License. -You may obtain a copy of the License at +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at - http://www.apache.org/licenses/LICENSE-2.0 + http://www.apache.org/licenses/LICENSE-2.0 -Unless required by applicable law or agreed to in writing, software -distributed under the License is distributed on an "AS IS" BASIS, -WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -See the License for the specific language governing permissions and -limitations under the License. - - */ +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ package org.apache.griffin.measure.rule.expr diff --git a/measure/src/main/scala/org/apache/griffin/measure/rule/expr/Cacheable.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/expr/Cacheable.scala index 1d94f0755..feb81569b 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rule/expr/Cacheable.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rule/expr/Cacheable.scala @@ -1,17 +1,21 @@ -/*- - * Licensed under the Apache License, Version 2.0 (the "License"); -you may not use this file except in compliance with the License. -You may obtain a copy of the License at +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at - http://www.apache.org/licenses/LICENSE-2.0 + http://www.apache.org/licenses/LICENSE-2.0 -Unless required by applicable law or agreed to in writing, software -distributed under the License is distributed on an "AS IS" BASIS, -WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -See the License for the specific language governing permissions and -limitations under the License. - - */ +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ package org.apache.griffin.measure.rule.expr trait Cacheable extends DataSourceable { diff --git a/measure/src/main/scala/org/apache/griffin/measure/rule/expr/Calculatable.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/expr/Calculatable.scala index 860e79ee6..904e823e4 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rule/expr/Calculatable.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rule/expr/Calculatable.scala @@ -1,17 +1,21 @@ -/*- - * Licensed under the Apache License, Version 2.0 (the "License"); -you may not use this file except in compliance with the License. -You may obtain a copy of the License at +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at - http://www.apache.org/licenses/LICENSE-2.0 + http://www.apache.org/licenses/LICENSE-2.0 -Unless required by applicable law or agreed to in writing, software -distributed under the License is distributed on an "AS IS" BASIS, -WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -See the License for the specific language governing permissions and -limitations under the License. - - */ +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ package org.apache.griffin.measure.rule.expr trait Calculatable extends Serializable { diff --git a/measure/src/main/scala/org/apache/griffin/measure/rule/expr/ClauseExpr.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/expr/ClauseExpr.scala index 7c412c2dd..a56e0db94 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rule/expr/ClauseExpr.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rule/expr/ClauseExpr.scala @@ -1,17 +1,21 @@ -/*- - * Licensed under the Apache License, Version 2.0 (the "License"); -you may not use this file except in compliance with the License. -You may obtain a copy of the License at +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at - http://www.apache.org/licenses/LICENSE-2.0 + http://www.apache.org/licenses/LICENSE-2.0 -Unless required by applicable law or agreed to in writing, software -distributed under the License is distributed on an "AS IS" BASIS, -WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -See the License for the specific language governing permissions and -limitations under the License. - - */ +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ package org.apache.griffin.measure.rule.expr diff --git a/measure/src/main/scala/org/apache/griffin/measure/rule/expr/DataSourceable.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/expr/DataSourceable.scala index 67bd2615f..e2cf17240 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rule/expr/DataSourceable.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rule/expr/DataSourceable.scala @@ -1,17 +1,21 @@ -/*- - * Licensed under the Apache License, Version 2.0 (the "License"); -you may not use this file except in compliance with the License. -You may obtain a copy of the License at +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at - http://www.apache.org/licenses/LICENSE-2.0 + http://www.apache.org/licenses/LICENSE-2.0 -Unless required by applicable law or agreed to in writing, software -distributed under the License is distributed on an "AS IS" BASIS, -WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -See the License for the specific language governing permissions and -limitations under the License. - - */ +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ package org.apache.griffin.measure.rule.expr trait DataSourceable extends Serializable { diff --git a/measure/src/main/scala/org/apache/griffin/measure/rule/expr/Describable.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/expr/Describable.scala index 0e6384fbb..393d7a6d3 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rule/expr/Describable.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rule/expr/Describable.scala @@ -1,17 +1,21 @@ -/*- - * Licensed under the Apache License, Version 2.0 (the "License"); -you may not use this file except in compliance with the License. -You may obtain a copy of the License at +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at - http://www.apache.org/licenses/LICENSE-2.0 + http://www.apache.org/licenses/LICENSE-2.0 -Unless required by applicable law or agreed to in writing, software -distributed under the License is distributed on an "AS IS" BASIS, -WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -See the License for the specific language governing permissions and -limitations under the License. - - */ +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ package org.apache.griffin.measure.rule.expr trait Describable extends Serializable { diff --git a/measure/src/main/scala/org/apache/griffin/measure/rule/expr/Expr.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/expr/Expr.scala index 8e67cd9e0..726b5b655 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rule/expr/Expr.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rule/expr/Expr.scala @@ -1,17 +1,21 @@ -/*- - * Licensed under the Apache License, Version 2.0 (the "License"); -you may not use this file except in compliance with the License. -You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, software -distributed under the License is distributed on an "AS IS" BASIS, -WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -See the License for the specific language governing permissions and -limitations under the License. - - */ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ package org.apache.griffin.measure.rule.expr import org.apache.spark.sql.types.DataType diff --git a/measure/src/main/scala/org/apache/griffin/measure/rule/expr/ExprDescOnly.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/expr/ExprDescOnly.scala index 17dab70c8..01b7e3cdf 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rule/expr/ExprDescOnly.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rule/expr/ExprDescOnly.scala @@ -1,17 +1,21 @@ -/*- - * Licensed under the Apache License, Version 2.0 (the "License"); -you may not use this file except in compliance with the License. -You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, software -distributed under the License is distributed on an "AS IS" BASIS, -WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -See the License for the specific language governing permissions and -limitations under the License. - - */ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ package org.apache.griffin.measure.rule.expr trait ExprDescOnly extends Describable { diff --git a/measure/src/main/scala/org/apache/griffin/measure/rule/expr/ExprIdCounter.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/expr/ExprIdCounter.scala index 790d6fede..ae76aefb6 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rule/expr/ExprIdCounter.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rule/expr/ExprIdCounter.scala @@ -1,17 +1,21 @@ -/*- - * Licensed under the Apache License, Version 2.0 (the "License"); -you may not use this file except in compliance with the License. -You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, software -distributed under the License is distributed on an "AS IS" BASIS, -WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -See the License for the specific language governing permissions and -limitations under the License. - - */ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ package org.apache.griffin.measure.rule.expr import java.util.concurrent.atomic.AtomicLong diff --git a/measure/src/main/scala/org/apache/griffin/measure/rule/expr/FieldDescOnly.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/expr/FieldDescOnly.scala index 3c10cfdd2..dca037b81 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rule/expr/FieldDescOnly.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rule/expr/FieldDescOnly.scala @@ -1,17 +1,21 @@ -/*- - * Licensed under the Apache License, Version 2.0 (the "License"); -you may not use this file except in compliance with the License. -You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, software -distributed under the License is distributed on an "AS IS" BASIS, -WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -See the License for the specific language governing permissions and -limitations under the License. - - */ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ package org.apache.griffin.measure.rule.expr import scala.util.{Success, Try} diff --git a/measure/src/main/scala/org/apache/griffin/measure/rule/expr/LiteralExpr.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/expr/LiteralExpr.scala index ecc9fd26b..5c78e03f7 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rule/expr/LiteralExpr.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rule/expr/LiteralExpr.scala @@ -1,17 +1,21 @@ -/*- - * Licensed under the Apache License, Version 2.0 (the "License"); -you may not use this file except in compliance with the License. -You may obtain a copy of the License at +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at - http://www.apache.org/licenses/LICENSE-2.0 + http://www.apache.org/licenses/LICENSE-2.0 -Unless required by applicable law or agreed to in writing, software -distributed under the License is distributed on an "AS IS" BASIS, -WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -See the License for the specific language governing permissions and -limitations under the License. - - */ +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ package org.apache.griffin.measure.rule.expr import org.apache.griffin.measure.utils.TimeUtil diff --git a/measure/src/main/scala/org/apache/griffin/measure/rule/expr/LogicalExpr.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/expr/LogicalExpr.scala index e9fa0da7e..a95e394a5 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rule/expr/LogicalExpr.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rule/expr/LogicalExpr.scala @@ -1,17 +1,21 @@ -/*- - * Licensed under the Apache License, Version 2.0 (the "License"); -you may not use this file except in compliance with the License. -You may obtain a copy of the License at +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at - http://www.apache.org/licenses/LICENSE-2.0 + http://www.apache.org/licenses/LICENSE-2.0 -Unless required by applicable law or agreed to in writing, software -distributed under the License is distributed on an "AS IS" BASIS, -WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -See the License for the specific language governing permissions and -limitations under the License. - - */ +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ package org.apache.griffin.measure.rule.expr import org.apache.griffin.measure.rule.CalculationUtil._ diff --git a/measure/src/main/scala/org/apache/griffin/measure/rule/expr/MathExpr.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/expr/MathExpr.scala index fc60b02a5..661e8f45c 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rule/expr/MathExpr.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rule/expr/MathExpr.scala @@ -1,17 +1,21 @@ -/*- - * Licensed under the Apache License, Version 2.0 (the "License"); -you may not use this file except in compliance with the License. -You may obtain a copy of the License at +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at - http://www.apache.org/licenses/LICENSE-2.0 + http://www.apache.org/licenses/LICENSE-2.0 -Unless required by applicable law or agreed to in writing, software -distributed under the License is distributed on an "AS IS" BASIS, -WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -See the License for the specific language governing permissions and -limitations under the License. - - */ +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ package org.apache.griffin.measure.rule.expr import org.apache.griffin.measure.rule.CalculationUtil._ diff --git a/measure/src/main/scala/org/apache/griffin/measure/rule/expr/SelectExpr.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/expr/SelectExpr.scala index 38ac8b8b9..760c63f6b 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rule/expr/SelectExpr.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rule/expr/SelectExpr.scala @@ -1,17 +1,21 @@ -/*- - * Licensed under the Apache License, Version 2.0 (the "License"); -you may not use this file except in compliance with the License. -You may obtain a copy of the License at +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at - http://www.apache.org/licenses/LICENSE-2.0 + http://www.apache.org/licenses/LICENSE-2.0 -Unless required by applicable law or agreed to in writing, software -distributed under the License is distributed on an "AS IS" BASIS, -WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -See the License for the specific language governing permissions and -limitations under the License. - - */ +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ package org.apache.griffin.measure.rule.expr import org.apache.spark.sql.types.DataType diff --git a/measure/src/main/scala/org/apache/griffin/measure/rule/func/DefaultFunctionDefine.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/func/DefaultFunctionDefine.scala index 2bfb42b65..341fb178f 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rule/func/DefaultFunctionDefine.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rule/func/DefaultFunctionDefine.scala @@ -1,17 +1,21 @@ -/*- - * Licensed under the Apache License, Version 2.0 (the "License"); -you may not use this file except in compliance with the License. -You may obtain a copy of the License at +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at - http://www.apache.org/licenses/LICENSE-2.0 + http://www.apache.org/licenses/LICENSE-2.0 -Unless required by applicable law or agreed to in writing, software -distributed under the License is distributed on an "AS IS" BASIS, -WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -See the License for the specific language governing permissions and -limitations under the License. - - */ +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ package org.apache.griffin.measure.rule.func import org.apache.griffin.measure.utils.JsonUtil diff --git a/measure/src/main/scala/org/apache/griffin/measure/rule/func/FunctionDefine.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/func/FunctionDefine.scala index 860d4c398..d23fc7af3 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rule/func/FunctionDefine.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rule/func/FunctionDefine.scala @@ -1,17 +1,21 @@ -/*- - * Licensed under the Apache License, Version 2.0 (the "License"); -you may not use this file except in compliance with the License. -You may obtain a copy of the License at +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at - http://www.apache.org/licenses/LICENSE-2.0 + http://www.apache.org/licenses/LICENSE-2.0 -Unless required by applicable law or agreed to in writing, software -distributed under the License is distributed on an "AS IS" BASIS, -WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -See the License for the specific language governing permissions and -limitations under the License. - - */ +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ package org.apache.griffin.measure.rule.func trait FunctionDefine extends Serializable { diff --git a/measure/src/main/scala/org/apache/griffin/measure/rule/func/FunctionUtil.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/func/FunctionUtil.scala index 623c8a07d..5451cb156 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rule/func/FunctionUtil.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rule/func/FunctionUtil.scala @@ -1,17 +1,21 @@ -/*- - * Licensed under the Apache License, Version 2.0 (the "License"); -you may not use this file except in compliance with the License. -You may obtain a copy of the License at +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at - http://www.apache.org/licenses/LICENSE-2.0 + http://www.apache.org/licenses/LICENSE-2.0 -Unless required by applicable law or agreed to in writing, software -distributed under the License is distributed on an "AS IS" BASIS, -WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -See the License for the specific language governing permissions and -limitations under the License. - - */ +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ package org.apache.griffin.measure.rule.func import java.lang.reflect.Method diff --git a/measure/src/main/scala/org/apache/griffin/measure/utils/HdfsUtil.scala b/measure/src/main/scala/org/apache/griffin/measure/utils/HdfsUtil.scala index 79b0742c9..89556846f 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/utils/HdfsUtil.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/utils/HdfsUtil.scala @@ -1,17 +1,21 @@ -/*- - * Licensed under the Apache License, Version 2.0 (the "License"); -you may not use this file except in compliance with the License. -You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, software -distributed under the License is distributed on an "AS IS" BASIS, -WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -See the License for the specific language governing permissions and -limitations under the License. - - */ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ package org.apache.griffin.measure.utils import org.apache.hadoop.conf.Configuration diff --git a/measure/src/main/scala/org/apache/griffin/measure/utils/HttpUtil.scala b/measure/src/main/scala/org/apache/griffin/measure/utils/HttpUtil.scala index be754a02a..d5a3d3f3a 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/utils/HttpUtil.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/utils/HttpUtil.scala @@ -1,17 +1,21 @@ -/*- - * Licensed under the Apache License, Version 2.0 (the "License"); -you may not use this file except in compliance with the License. -You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, software -distributed under the License is distributed on an "AS IS" BASIS, -WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -See the License for the specific language governing permissions and -limitations under the License. - - */ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ package org.apache.griffin.measure.utils import scalaj.http._ diff --git a/measure/src/main/scala/org/apache/griffin/measure/utils/JsonUtil.scala b/measure/src/main/scala/org/apache/griffin/measure/utils/JsonUtil.scala index c5218350f..76d5d6d89 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/utils/JsonUtil.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/utils/JsonUtil.scala @@ -1,17 +1,21 @@ -/*- - * Licensed under the Apache License, Version 2.0 (the "License"); -you may not use this file except in compliance with the License. -You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, software -distributed under the License is distributed on an "AS IS" BASIS, -WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -See the License for the specific language governing permissions and -limitations under the License. - - */ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ package org.apache.griffin.measure.utils import java.io.InputStream diff --git a/measure/src/main/scala/org/apache/griffin/measure/utils/TimeUtil.scala b/measure/src/main/scala/org/apache/griffin/measure/utils/TimeUtil.scala index c916f19ec..fa8a0ea16 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/utils/TimeUtil.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/utils/TimeUtil.scala @@ -1,17 +1,21 @@ -/*- - * Licensed under the Apache License, Version 2.0 (the "License"); -you may not use this file except in compliance with the License. -You may obtain a copy of the License at +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at - http://www.apache.org/licenses/LICENSE-2.0 + http://www.apache.org/licenses/LICENSE-2.0 -Unless required by applicable law or agreed to in writing, software -distributed under the License is distributed on an "AS IS" BASIS, -WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -See the License for the specific language governing permissions and -limitations under the License. - - */ +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ package org.apache.griffin.measure.utils import scala.util.{Failure, Success, Try} diff --git a/measure/src/test/resources/config-streaming.json b/measure/src/test/resources/config-streaming.json new file mode 100644 index 000000000..4a355486b --- /dev/null +++ b/measure/src/test/resources/config-streaming.json @@ -0,0 +1,57 @@ +{ + "name": "accu2", + "type": "accuracy", + + "source": { + "type": "kafka", + "version": "0.8", + "config": { + "kafka.config": { + "bootstrap.servers": "a.b.c.d:9092", + "group.id": "group1", + "auto.offset.reset": "smallest", + "auto.commit.enable": "false", + }, + "topics": "src", + "key.type": "java.lang.String", + "value.type": "java.lang.String", + "cache": { + "type": "temp", + "config": { + "table.name": "source", + "info.path": "src" + } + } + } + }, + + "target": { + "type": "kafka", + "version": "0.8", + "config": { + "kafka.config": { + "bootstrap.servers": "a.b.c.d:9092", + "group.id": "group1", + "auto.offset.reset": "smallest", + "auto.commit.enable": "false", + }, + "topics": "tgt", + "key.type": "java.lang.String", + "value.type": "java.lang.String", + "cache": { + "type": "hive", + "version": 1.2, + "config": { + "database": "default", + "table.name": "target_table", + "info.path": "tgt" + } + } + } + }, + + "evaluateRule": { + "sampleRatio": 0.2, + "rules": "$source.json().seeds[*].json().url = $target.json().groups[0].attrsList['name' = 'URL'].values[0] AND $source.json().seeds[*].json().metadata.json().tracker.crawlRequestCreateTS = $target.json().groups[0].attrsList['name' = 'CRAWLMETADATA'].values[0].json().tracker.crawlRequestCreateTS WHEN $source._timestamp_ + 24h < $target._timestamp_" + } +} \ No newline at end of file diff --git a/measure/src/test/resources/env.json b/measure/src/test/resources/env.json index 3a9e38c61..14e3b75a2 100644 --- a/measure/src/test/resources/env.json +++ b/measure/src/test/resources/env.json @@ -1,7 +1,9 @@ { "spark": { - "log.level": "ERROR", + "log.level": "INFO", "checkpoint.dir": "hdfs:///griffin/batch/cp", + "batch.interval": "10s", + "process.interval": "10m", "config": {} }, @@ -9,14 +11,30 @@ { "type": "hdfs", "config": { - "path": "hdfs:///griffin/streaming/persist" + "path": "hdfs:///griffin/streaming/persist", + "max.persist.lines": 10000, + "max.lines.per.file": 10000 } }, { "type": "http", "config": { "method": "post", - "api": "http://phxbark4dq-360935.stratus.phx.ebay.com:8080/" + "api": "http://HOSTNAME:9200/griffin/accuracy" + } + } + ], + + "info.cache": [ + { + "type": "zk", + "config": { + "hosts": "localhost:2181", + "namespace": "griffin/infocache", + "lock.path": "lock", + "mode": "persist", + "init.clear": true, + "close.clear": false } } ], diff --git a/measure/src/test/scala/org/apache/griffin/measure/algo/BatchAccuracyAlgoTest.scala b/measure/src/test/scala/org/apache/griffin/measure/algo/batch/BatchAccuracyAlgoTest.scala similarity index 89% rename from measure/src/test/scala/org/apache/griffin/measure/algo/BatchAccuracyAlgoTest.scala rename to measure/src/test/scala/org/apache/griffin/measure/algo/batch/BatchAccuracyAlgoTest.scala index 57a908e20..70d27f62b 100644 --- a/measure/src/test/scala/org/apache/griffin/measure/algo/BatchAccuracyAlgoTest.scala +++ b/measure/src/test/scala/org/apache/griffin/measure/algo/batch/BatchAccuracyAlgoTest.scala @@ -1,18 +1,22 @@ -/*- - * Licensed under the Apache License, Version 2.0 (the "License"); -you may not use this file except in compliance with the License. -You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, software -distributed under the License is distributed on an "AS IS" BASIS, -WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -See the License for the specific language governing permissions and -limitations under the License. - - */ - package org.apache.griffin.measure.algo +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ +package org.apache.griffin.measure.algo import java.util.Date diff --git a/measure/src/test/scala/org/apache/griffin/measure/algo/BatchProfileAlgoTest.scala b/measure/src/test/scala/org/apache/griffin/measure/algo/batch/BatchProfileAlgoTest.scala similarity index 86% rename from measure/src/test/scala/org/apache/griffin/measure/algo/BatchProfileAlgoTest.scala rename to measure/src/test/scala/org/apache/griffin/measure/algo/batch/BatchProfileAlgoTest.scala index be419a585..8e11c0a71 100644 --- a/measure/src/test/scala/org/apache/griffin/measure/algo/BatchProfileAlgoTest.scala +++ b/measure/src/test/scala/org/apache/griffin/measure/algo/batch/BatchProfileAlgoTest.scala @@ -1,17 +1,21 @@ -///*- -// * Licensed under the Apache License, Version 2.0 (the "License"); -//you may not use this file except in compliance with the License. -//You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -//Unless required by applicable law or agreed to in writing, software -//distributed under the License is distributed on an "AS IS" BASIS, -//WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -//See the License for the specific language governing permissions and -//limitations under the License. -// -// */ +///* +//Licensed to the Apache Software Foundation (ASF) under one +//or more contributor license agreements. See the NOTICE file +//distributed with this work for additional information +//regarding copyright ownership. The ASF licenses this file +//to you under the Apache License, Version 2.0 (the +//"License"); you may not use this file except in compliance +//with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +//Unless required by applicable law or agreed to in writing, +//software distributed under the License is distributed on an +//"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +//KIND, either express or implied. See the License for the +//specific language governing permissions and limitations +//under the License. +//*/ //package org.apache.griffin.measure.algo // //import java.util.Date diff --git a/measure/src/test/scala/org/apache/griffin/measure/algo/core/AccuracyCoreTest.scala b/measure/src/test/scala/org/apache/griffin/measure/algo/core/AccuracyCoreTest.scala index 5b3c73cf7..2179fbabf 100644 --- a/measure/src/test/scala/org/apache/griffin/measure/algo/core/AccuracyCoreTest.scala +++ b/measure/src/test/scala/org/apache/griffin/measure/algo/core/AccuracyCoreTest.scala @@ -1,17 +1,21 @@ -/*- - * Licensed under the Apache License, Version 2.0 (the "License"); -you may not use this file except in compliance with the License. -You may obtain a copy of the License at +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at - http://www.apache.org/licenses/LICENSE-2.0 + http://www.apache.org/licenses/LICENSE-2.0 -Unless required by applicable law or agreed to in writing, software -distributed under the License is distributed on an "AS IS" BASIS, -WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -See the License for the specific language governing permissions and -limitations under the License. - - */ +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ package org.apache.griffin.measure.algo.core import org.apache.griffin.measure.config.params.user.EvaluateRuleParam diff --git a/measure/src/test/scala/org/apache/griffin/measure/algo/core/ProfileCoreTest.scala b/measure/src/test/scala/org/apache/griffin/measure/algo/core/ProfileCoreTest.scala index 1a697f8bd..087e8e59e 100644 --- a/measure/src/test/scala/org/apache/griffin/measure/algo/core/ProfileCoreTest.scala +++ b/measure/src/test/scala/org/apache/griffin/measure/algo/core/ProfileCoreTest.scala @@ -1,17 +1,21 @@ -/*- - * Licensed under the Apache License, Version 2.0 (the "License"); -you may not use this file except in compliance with the License. -You may obtain a copy of the License at +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at - http://www.apache.org/licenses/LICENSE-2.0 + http://www.apache.org/licenses/LICENSE-2.0 -Unless required by applicable law or agreed to in writing, software -distributed under the License is distributed on an "AS IS" BASIS, -WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -See the License for the specific language governing permissions and -limitations under the License. - - */ +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ package org.apache.griffin.measure.algo.core import org.apache.griffin.measure.config.params.user.EvaluateRuleParam diff --git a/measure/src/test/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgoTest.scala b/measure/src/test/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgoTest.scala new file mode 100644 index 000000000..df33b75a4 --- /dev/null +++ b/measure/src/test/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgoTest.scala @@ -0,0 +1,239 @@ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ +package org.apache.griffin.measure.algo.streaming + +import java.util.Date +import java.util.concurrent.TimeUnit + +import org.apache.griffin.measure.algo.batch.BatchAccuracyAlgo +import org.apache.griffin.measure.cache.InfoCacheInstance +import org.apache.griffin.measure.config.params._ +import org.apache.griffin.measure.config.params.env._ +import org.apache.griffin.measure.config.params.user._ +import org.apache.griffin.measure.config.reader._ +import org.apache.griffin.measure.config.validator._ +import org.apache.griffin.measure.connector.{BatchDataConnector, DataConnector, DataConnectorFactory} +import org.apache.griffin.measure.log.Loggable +import org.apache.griffin.measure.persist.{Persist, PersistFactory} +import org.apache.griffin.measure.rule.expr._ +import org.apache.griffin.measure.rule.{ExprValueUtil, RuleAnalyzer, RuleFactory} +import org.apache.griffin.measure.utils.TimeUtil +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.SQLContext +import org.apache.spark.streaming.{Milliseconds, StreamingContext} +import org.apache.spark.{SparkConf, SparkContext} +import org.junit.runner.RunWith +import org.scalatest.junit.JUnitRunner +import org.scalatest.{BeforeAndAfter, FunSuite, Matchers} + +import scala.util.{Failure, Success, Try} + + +@RunWith(classOf[JUnitRunner]) +class StreamingAccuracyAlgoTest extends FunSuite with Matchers with BeforeAndAfter with Loggable { + + val envFile = "src/test/resources/env.json" + val confFile = "src/test/resources/config-streaming.json" + val envFsType = "local" + val userFsType = "local" + + val args = Array(envFile, confFile) + + var sc: SparkContext = _ + var sqlContext: SQLContext = _ +// val ssc: StreamingContext = _ + + var allParam: AllParam = _ + + before { + // read param files + val envParam = readParamFile[EnvParam](envFile, envFsType) match { + case Success(p) => p + case Failure(ex) => { + error(ex.getMessage) + sys.exit(-2) + } + } + val userParam = readParamFile[UserParam](confFile, userFsType) match { + case Success(p) => p + case Failure(ex) => { + error(ex.getMessage) + sys.exit(-2) + } + } + allParam = AllParam(envParam, userParam) + + // validate param files + validateParams(allParam) match { + case Failure(ex) => { + error(ex.getMessage) + sys.exit(-3) + } + case _ => { + info("params validation pass") + } + } + + val metricName = userParam.name + val conf = new SparkConf().setMaster("local[*]").setAppName(metricName) + sc = new SparkContext(conf) + sqlContext = new SQLContext(sc) + } + + test("algorithm") { + val envParam = allParam.envParam + val userParam = allParam.userParam + val metricName = userParam.name + val sparkParam = envParam.sparkParam + + val batchInterval = TimeUtil.milliseconds(sparkParam.batchInterval) match { + case Some(interval) => Milliseconds(interval) + case _ => throw new Exception("invalid batch interval") + } + val ssc = new StreamingContext(sc, batchInterval) + ssc.checkpoint(sparkParam.cpDir) + + // start time + val startTime = new Date().getTime() + + // get persists to persist measure result +// val persist: Persist = PersistFactory(envParam.persistParams, metricName).getPersists(startTime) + + // get spark application id + val applicationId = sc.applicationId + + // persist start id +// persist.start(applicationId) + + // generate rule from rule param, generate rule analyzer + val ruleFactory = RuleFactory(userParam.evaluateRuleParam) + val rule: StatementExpr = ruleFactory.generateRule() + val ruleAnalyzer: RuleAnalyzer = RuleAnalyzer(rule) + + // const expr value map + val constExprValueMap = ExprValueUtil.genExprValueMaps(None, ruleAnalyzer.constCacheExprs, Map[String, Any]()) + val finalConstExprValueMap = ExprValueUtil.updateExprValueMaps(ruleAnalyzer.constFinalCacheExprs, constExprValueMap) + val finalConstMap = finalConstExprValueMap.headOption match { + case Some(m) => m + case _ => Map[String, Any]() + } + + // data connector + val sourceDataConnector: BatchDataConnector = + DataConnectorFactory.getDataConnector(sqlContext, ssc, userParam.sourceParam, + ruleAnalyzer.sourceRuleExprs, finalConstMap + ) match { + case Success(cntr) => { + if (cntr.available) cntr + else throw new Exception("source data connection error!") + } + case Failure(ex) => throw ex + } + val targetDataConnector: BatchDataConnector = + DataConnectorFactory.getDataConnector(sqlContext, ssc, userParam.targetParam, + ruleAnalyzer.targetRuleExprs, finalConstMap + ) match { + case Success(cntr) => { + if (cntr.available) cntr + else throw new Exception("target data connection error!") + } + case Failure(ex) => throw ex + } + + InfoCacheInstance.initInstance(envParam.infoCacheParams, metricName) + InfoCacheInstance.init + + // my algo + val algo = StreamingAccuracyAlgo(allParam) + + // process thread + case class Process() extends Runnable { + val lock = InfoCacheInstance.genLock("process") + def run(): Unit = { + val locked = lock.lock(5, TimeUnit.SECONDS) + if (locked) { + try { + val st = new Date().getTime + // get data + val sourceData = sourceDataConnector.data match { + case Success(dt) => dt + case Failure(ex) => throw ex + } + val targetData = targetDataConnector.data match { + case Success(dt) => dt + case Failure(ex) => throw ex + } + + // accuracy algorithm + val (accuResult, missingRdd, matchedRdd) = algo.accuracy(sourceData, targetData, ruleAnalyzer) + + println(accuResult) + + val et = new Date().getTime + + // persist time + // persist.log(et, s"calculation using time: ${et - st} ms") + + // persist result + // persist.result(et, accuResult) + val missingRecords = missingRdd.map(algo.record2String(_, ruleAnalyzer.sourceRuleExprs.persistExprs, ruleAnalyzer.targetRuleExprs.persistExprs)) + // persist.missRecords(missingRecords) + + // val pet = new Date().getTime + // persist.log(pet, s"persist using time: ${pet - et} ms") + } finally { + lock.unlock() + } + } + } + } + + val processInterval = TimeUtil.milliseconds(sparkParam.processInterval) match { + case Some(interval) => interval + case _ => throw new Exception("invalid batch interval") + } + val process = StreamingProcess(processInterval, Process()) + + process.startup() + + ssc.start() + ssc.awaitTermination() + ssc.stop(stopSparkContext=true, stopGracefully=true) + + // context stop + sc.stop + + InfoCacheInstance.close + +// persist.finish() + + process.shutdown() + } + + private def readParamFile[T <: Param](file: String, fsType: String)(implicit m : Manifest[T]): Try[T] = { + val paramReader = ParamReaderFactory.getParamReader(file, fsType) + paramReader.readConfig[T] + } + + private def validateParams(allParam: AllParam): Try[Boolean] = { + val allParamValidator = AllParamValidator() + allParamValidator.validate(allParam) + } + +} diff --git a/measure/src/test/scala/org/apache/griffin/measure/cache/InfoCacheInstanceTest.scala b/measure/src/test/scala/org/apache/griffin/measure/cache/InfoCacheInstanceTest.scala index bd99fb141..1458f0960 100644 --- a/measure/src/test/scala/org/apache/griffin/measure/cache/InfoCacheInstanceTest.scala +++ b/measure/src/test/scala/org/apache/griffin/measure/cache/InfoCacheInstanceTest.scala @@ -1,17 +1,21 @@ -/*- - * Licensed under the Apache License, Version 2.0 (the "License"); -you may not use this file except in compliance with the License. -You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, software -distributed under the License is distributed on an "AS IS" BASIS, -WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -See the License for the specific language governing permissions and -limitations under the License. - - */ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ package org.apache.griffin.measure.cache import java.util.Date diff --git a/measure/src/test/scala/org/apache/griffin/measure/cache/ZKCacheLockTest.scala b/measure/src/test/scala/org/apache/griffin/measure/cache/ZKCacheLockTest.scala index f016ac6e1..8e3dffc35 100644 --- a/measure/src/test/scala/org/apache/griffin/measure/cache/ZKCacheLockTest.scala +++ b/measure/src/test/scala/org/apache/griffin/measure/cache/ZKCacheLockTest.scala @@ -1,17 +1,21 @@ -/*- - * Licensed under the Apache License, Version 2.0 (the "License"); -you may not use this file except in compliance with the License. -You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, software -distributed under the License is distributed on an "AS IS" BASIS, -WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -See the License for the specific language governing permissions and -limitations under the License. - - */ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ package org.apache.griffin.measure.cache import java.util.Date diff --git a/measure/src/test/scala/org/apache/griffin/measure/cache/ZKInfoCacheTest.scala b/measure/src/test/scala/org/apache/griffin/measure/cache/ZKInfoCacheTest.scala index f1a4c382b..5e9795713 100644 --- a/measure/src/test/scala/org/apache/griffin/measure/cache/ZKInfoCacheTest.scala +++ b/measure/src/test/scala/org/apache/griffin/measure/cache/ZKInfoCacheTest.scala @@ -1,17 +1,21 @@ -/*- - * Licensed under the Apache License, Version 2.0 (the "License"); -you may not use this file except in compliance with the License. -You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, software -distributed under the License is distributed on an "AS IS" BASIS, -WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -See the License for the specific language governing permissions and -limitations under the License. - - */ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ package org.apache.griffin.measure.cache import java.util.Date diff --git a/measure/src/test/scala/org/apache/griffin/measure/config/reader/ParamRawStringReaderTest.scala b/measure/src/test/scala/org/apache/griffin/measure/config/reader/ParamRawStringReaderTest.scala index 757c6403d..b3c94e56f 100644 --- a/measure/src/test/scala/org/apache/griffin/measure/config/reader/ParamRawStringReaderTest.scala +++ b/measure/src/test/scala/org/apache/griffin/measure/config/reader/ParamRawStringReaderTest.scala @@ -1,17 +1,21 @@ -/*- - * Licensed under the Apache License, Version 2.0 (the "License"); -you may not use this file except in compliance with the License. -You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, software -distributed under the License is distributed on an "AS IS" BASIS, -WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -See the License for the specific language governing permissions and -limitations under the License. - - */ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ package org.apache.griffin.measure.config.reader import org.apache.griffin.measure.config.params.env._ diff --git a/measure/src/test/scala/org/apache/griffin/measure/config/validator/AllParamValidatorTest.scala b/measure/src/test/scala/org/apache/griffin/measure/config/validator/AllParamValidatorTest.scala index 12963054e..8000c6593 100644 --- a/measure/src/test/scala/org/apache/griffin/measure/config/validator/AllParamValidatorTest.scala +++ b/measure/src/test/scala/org/apache/griffin/measure/config/validator/AllParamValidatorTest.scala @@ -1,17 +1,21 @@ -/*- - * Licensed under the Apache License, Version 2.0 (the "License"); -you may not use this file except in compliance with the License. -You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, software -distributed under the License is distributed on an "AS IS" BASIS, -WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -See the License for the specific language governing permissions and -limitations under the License. - - */ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ package org.apache.griffin.measure.config.validator import org.apache.griffin.measure.config.params._ diff --git a/measure/src/test/scala/org/apache/griffin/measure/connector/ConnectorTest.scala b/measure/src/test/scala/org/apache/griffin/measure/connector/ConnectorTest.scala index d4f83e09c..228ec2689 100644 --- a/measure/src/test/scala/org/apache/griffin/measure/connector/ConnectorTest.scala +++ b/measure/src/test/scala/org/apache/griffin/measure/connector/ConnectorTest.scala @@ -1,17 +1,21 @@ -/*- - * Licensed under the Apache License, Version 2.0 (the "License"); -you may not use this file except in compliance with the License. -You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, software -distributed under the License is distributed on an "AS IS" BASIS, -WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -See the License for the specific language governing permissions and -limitations under the License. - - */ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ package org.apache.griffin.measure.connector import org.apache.griffin.measure.config.params.env._ diff --git a/measure/src/test/scala/org/apache/griffin/measure/persist/HdfsPersistTest.scala b/measure/src/test/scala/org/apache/griffin/measure/persist/HdfsPersistTest.scala index e9d3f92b3..2dfceb32a 100644 --- a/measure/src/test/scala/org/apache/griffin/measure/persist/HdfsPersistTest.scala +++ b/measure/src/test/scala/org/apache/griffin/measure/persist/HdfsPersistTest.scala @@ -1,17 +1,21 @@ -/*- - * Licensed under the Apache License, Version 2.0 (the "License"); -you may not use this file except in compliance with the License. -You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, software -distributed under the License is distributed on an "AS IS" BASIS, -WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -See the License for the specific language governing permissions and -limitations under the License. - - */ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ package org.apache.griffin.measure.persist import org.junit.runner.RunWith diff --git a/measure/src/test/scala/org/apache/griffin/measure/persist/HttpPersistTest.scala b/measure/src/test/scala/org/apache/griffin/measure/persist/HttpPersistTest.scala index 14b9e3ec0..1b0bc6f0a 100644 --- a/measure/src/test/scala/org/apache/griffin/measure/persist/HttpPersistTest.scala +++ b/measure/src/test/scala/org/apache/griffin/measure/persist/HttpPersistTest.scala @@ -1,17 +1,21 @@ -/*- - * Licensed under the Apache License, Version 2.0 (the "License"); -you may not use this file except in compliance with the License. -You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, software -distributed under the License is distributed on an "AS IS" BASIS, -WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -See the License for the specific language governing permissions and -limitations under the License. - - */ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ package org.apache.griffin.measure.persist import org.junit.runner.RunWith diff --git a/measure/src/test/scala/org/apache/griffin/measure/result/AccuracyResultTest.scala b/measure/src/test/scala/org/apache/griffin/measure/result/AccuracyResultTest.scala index 33e59c350..8a455300e 100644 --- a/measure/src/test/scala/org/apache/griffin/measure/result/AccuracyResultTest.scala +++ b/measure/src/test/scala/org/apache/griffin/measure/result/AccuracyResultTest.scala @@ -1,17 +1,21 @@ -/*- - * Licensed under the Apache License, Version 2.0 (the "License"); -you may not use this file except in compliance with the License. -You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, software -distributed under the License is distributed on an "AS IS" BASIS, -WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -See the License for the specific language governing permissions and -limitations under the License. - - */ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ package org.apache.griffin.measure.result import org.junit.runner.RunWith diff --git a/measure/src/test/scala/org/apache/griffin/measure/result/ProfileResultTest.scala b/measure/src/test/scala/org/apache/griffin/measure/result/ProfileResultTest.scala index 10fc33d77..2c4fc5f6a 100644 --- a/measure/src/test/scala/org/apache/griffin/measure/result/ProfileResultTest.scala +++ b/measure/src/test/scala/org/apache/griffin/measure/result/ProfileResultTest.scala @@ -1,17 +1,21 @@ -/*- - * Licensed under the Apache License, Version 2.0 (the "License"); -you may not use this file except in compliance with the License. -You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, software -distributed under the License is distributed on an "AS IS" BASIS, -WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -See the License for the specific language governing permissions and -limitations under the License. - - */ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ package org.apache.griffin.measure.result import org.junit.runner.RunWith diff --git a/measure/src/test/scala/org/apache/griffin/measure/rule/RuleAnalyzerTest.scala b/measure/src/test/scala/org/apache/griffin/measure/rule/RuleAnalyzerTest.scala index cb0438034..5b1d1eee6 100644 --- a/measure/src/test/scala/org/apache/griffin/measure/rule/RuleAnalyzerTest.scala +++ b/measure/src/test/scala/org/apache/griffin/measure/rule/RuleAnalyzerTest.scala @@ -1,17 +1,21 @@ -/*- - * Licensed under the Apache License, Version 2.0 (the "License"); -you may not use this file except in compliance with the License. -You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, software -distributed under the License is distributed on an "AS IS" BASIS, -WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -See the License for the specific language governing permissions and -limitations under the License. - - */ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ package org.apache.griffin.measure.rule import org.apache.griffin.measure.config.params.user.EvaluateRuleParam diff --git a/measure/src/test/scala/org/apache/griffin/measure/rule/RuleFactoryTest.scala b/measure/src/test/scala/org/apache/griffin/measure/rule/RuleFactoryTest.scala index 7ca1cb35c..c14cd0456 100644 --- a/measure/src/test/scala/org/apache/griffin/measure/rule/RuleFactoryTest.scala +++ b/measure/src/test/scala/org/apache/griffin/measure/rule/RuleFactoryTest.scala @@ -1,17 +1,21 @@ -/*- - * Licensed under the Apache License, Version 2.0 (the "License"); -you may not use this file except in compliance with the License. -You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, software -distributed under the License is distributed on an "AS IS" BASIS, -WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -See the License for the specific language governing permissions and -limitations under the License. - - */ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ package org.apache.griffin.measure.rule import org.apache.griffin.measure.config.params.user.EvaluateRuleParam diff --git a/measure/src/test/scala/org/apache/griffin/measure/rule/RuleParserTest.scala b/measure/src/test/scala/org/apache/griffin/measure/rule/RuleParserTest.scala index cbc95e145..ef57fb1d8 100644 --- a/measure/src/test/scala/org/apache/griffin/measure/rule/RuleParserTest.scala +++ b/measure/src/test/scala/org/apache/griffin/measure/rule/RuleParserTest.scala @@ -1,17 +1,21 @@ -/*- - * Licensed under the Apache License, Version 2.0 (the "License"); -you may not use this file except in compliance with the License. -You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, software -distributed under the License is distributed on an "AS IS" BASIS, -WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -See the License for the specific language governing permissions and -limitations under the License. - - */ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ package org.apache.griffin.measure.rule import org.junit.runner.RunWith diff --git a/measure/src/test/scala/org/apache/griffin/measure/utils/JsonUtilTest.scala b/measure/src/test/scala/org/apache/griffin/measure/utils/JsonUtilTest.scala index 9414d87d3..4daebb6b7 100644 --- a/measure/src/test/scala/org/apache/griffin/measure/utils/JsonUtilTest.scala +++ b/measure/src/test/scala/org/apache/griffin/measure/utils/JsonUtilTest.scala @@ -1,17 +1,21 @@ -/*- - * Licensed under the Apache License, Version 2.0 (the "License"); -you may not use this file except in compliance with the License. -You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, software -distributed under the License is distributed on an "AS IS" BASIS, -WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -See the License for the specific language governing permissions and -limitations under the License. - - */ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ package org.apache.griffin.measure.utils import org.junit.runner.RunWith From 40077f656a29798cbe1f84066e9db18f431edb65 Mon Sep 17 00:00:00 2001 From: Lionel Liu Date: Mon, 10 Jul 2017 18:20:51 +0800 Subject: [PATCH 014/111] t3 --- .../connector/DataConnectorFactory.scala | 7 + .../connector/KafkaDataConnector.scala | 3 +- .../KafkaStreamingDataConnector.scala | 2 +- .../connector/StreamingDataConnector.scala | 2 +- .../griffin/measure/rule/ExprValueUtil.scala | 112 +----- .../rule/func/DefaultFunctionDefine.scala | 7 +- .../measure/rule/func/FunctionUtil.scala | 23 +- .../src/test/resources/config-streaming.json | 8 +- measure/src/test/resources/env-streaming.json | 45 +++ .../algo/batch/BatchProfileAlgoTest.scala | 339 +++++++++--------- .../measure/connector/ConnectorTest.scala | 200 +++++++++++ 11 files changed, 461 insertions(+), 287 deletions(-) create mode 100644 measure/src/test/resources/env-streaming.json diff --git a/measure/src/main/scala/org/apache/griffin/measure/connector/DataConnectorFactory.scala b/measure/src/main/scala/org/apache/griffin/measure/connector/DataConnectorFactory.scala index 962d4f6ad..c1a1caeb3 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/connector/DataConnectorFactory.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/connector/DataConnectorFactory.scala @@ -109,6 +109,13 @@ object DataConnectorFactory { val ValueType = "value.type" val keyType = config.getOrElse(KeyType, "java.lang.String").toString val valueType = config.getOrElse(ValueType, "java.lang.String").toString + val KafkaConfig = "kafka.config" + val Topics = "topics" + val kafkaConfig = config.get(KafkaConfig) match { + case Some(map: Map[String, Any]) => map.mapValues(_.toString) + case _ => Map[String, String]() + } + val topics = config.getOrElse(Topics, "").toString (getClassTag(keyType), getClassTag(valueType)) match { case (ClassTag(k: Class[String]), ClassTag(v: Class[String])) => { new KafkaStreamingDataConnector(ssc, config) { diff --git a/measure/src/main/scala/org/apache/griffin/measure/connector/KafkaDataConnector.scala b/measure/src/main/scala/org/apache/griffin/measure/connector/KafkaDataConnector.scala index 1a882da19..22adb82db 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/connector/KafkaDataConnector.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/connector/KafkaDataConnector.scala @@ -78,7 +78,8 @@ case class KafkaDataConnector(sqlContext: SQLContext, ssc: StreamingContext, dat // } def available(): Boolean = { - kafkaStreamingDataConnector.available && cacheDataConnector.available +// kafkaStreamingDataConnector.available && cacheDataConnector.available + cacheDataConnector.available } override def init(): Unit = { diff --git a/measure/src/main/scala/org/apache/griffin/measure/connector/KafkaStreamingDataConnector.scala b/measure/src/main/scala/org/apache/griffin/measure/connector/KafkaStreamingDataConnector.scala index 8a02601c2..b7f932be7 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/connector/KafkaStreamingDataConnector.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/connector/KafkaStreamingDataConnector.scala @@ -35,7 +35,7 @@ abstract class KafkaStreamingDataConnector(ssc: StreamingContext, config: Map[St val Topics = "topics" val kafkaConfig = config.get(KafkaConfig) match { - case map: Map[String, Any] => map.mapValues(_.toString) + case Some(map: Map[String, Any]) => map.mapValues(_.toString).map(identity) case _ => Map[String, String]() } val topics = config.getOrElse(Topics, "").toString diff --git a/measure/src/main/scala/org/apache/griffin/measure/connector/StreamingDataConnector.scala b/measure/src/main/scala/org/apache/griffin/measure/connector/StreamingDataConnector.scala index 9e6330cbd..a686703fa 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/connector/StreamingDataConnector.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/connector/StreamingDataConnector.scala @@ -23,7 +23,7 @@ import org.apache.spark.streaming.dstream.InputDStream import scala.util.Try -trait StreamingDataConnector extends DataConnector { +trait StreamingDataConnector { type K type V diff --git a/measure/src/main/scala/org/apache/griffin/measure/rule/ExprValueUtil.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/ExprValueUtil.scala index 7d7459c0a..8fdd81e25 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rule/ExprValueUtil.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rule/ExprValueUtil.scala @@ -26,79 +26,6 @@ import scala.util.{Success, Try} object ExprValueUtil { -// private def calcExprValue(originDatas: Seq[Option[Any]], expr: Expr, existExprValueMap: Map[String, Any]): Seq[Option[Any]] = { -// originDatas.flatMap { originData => -// calcExprValue(originData, expr, existExprValueMap) -// } -// } - - // from origin data such as a Row of DataFrame, with existed expr value map, calculate related expression, get the expression value - // for now, one expr only get one value, not supporting one expr get multiple values - // params: - // - originData: the origin data such as a Row of DataFrame - // - expr: the expression to be calculated - // - existExprValueMap: existed expression value map, which might be used to get some existed expression value during calculation - // output: the calculated expression values -// private def calcExprValue(originData: Option[Any], expr: Expr, existExprValueMap: Map[String, Any]): Seq[Option[Any]] = { -// Try { -// expr match { -// case selection: SelectionExpr => { -// selection.selectors.foldLeft(Seq(originData)) { (datas, selector) => -// calcExprValue(datas, selector, existExprValueMap) -// } -// } -// case selector: IndexFieldRangeSelectExpr => { -// originData match { -// case Some(row: Row) => { -// if (selector.fields.size == 1) { -// selector.fields.head match { -// case i: IndexDesc => Seq(Some(row.getAs[Any](i.index))) -// case f: FieldDesc => Seq(Some(row.getAs[Any](f.field))) -// case _ => Nil -// } -// } else Nil -// } -// case Some(d: Map[String, Any]) => { -// selector.fields.foldLeft(Seq[Option[Any]]()) { (results, field) => -// results ++ (field match { -// case f: FieldDesc => opt2Seq(d.get(f.field)) -// case a: AllFieldsDesc => d.values.map(Some(_)).toSeq -// case _ => Nil -// }) -// } -// } -// case Some(d: Seq[Any]) => { -// selector.fields.foldLeft(Seq[Option[Any]]()) { (results, field) => -// results ++ (field match { -// case i: IndexDesc => opt2Seq(try { Some(d(i.index)) } catch { case _ => None }) -// case a: AllFieldsDesc => d.map(Some(_)) -// case r: FieldRangeDesc => Nil // not done -// case _ => Nil -// }) -// } -// } -// case _ => Nil -// } -// } -// case selector: FunctionOperationExpr => { -// val args: Array[Option[Any]] = selector.args.map { arg => -// arg.calculate(existExprValueMap) -// }.toArray -// originData match { -// case Some(d: String) => { -// FunctionUtil.invoke(selector.func, Some(d) +: args) -// } -// case _ => Nil -// } -// } -// case _ => Seq(expr.calculate(existExprValueMap)) -// } -// } match { -// case Success(v) => v -// case _ => Nil -// } -// } - private def append(path: List[String], step: String): List[String] = { path :+ step } @@ -204,7 +131,7 @@ object ExprValueUtil { } private def calcExprsValues(data: Option[Any], exprs: Iterable[Expr], existExprValueMap: Map[String, Any]): List[Map[String, Any]] = { - val schemaValues: Map[String, List[(List[String], Any)]] = exprs.map { expr => + val selectionValues: Map[String, List[(List[String], Any)]] = exprs.map { expr => (expr._id, calcExprValues((Nil, data) :: Nil, expr, existExprValueMap).flatMap { pair => pair._2 match { case Some(v) => Some((pair._1, v)) @@ -212,43 +139,28 @@ object ExprValueUtil { } }) }.toMap - SchemaValueCombineUtil.cartesian(schemaValues) + SchemaValueCombineUtil.cartesian(selectionValues) } - // try to calculate expr from data and initExprValueMap, generate new expression value maps - // depends on origin data and existed expr value map -// def genExprValueMap(data: Option[Any], expr: Expr, initExprValueMap: Map[String, Any]): Seq[Map[String, Any]] = { -// val valueOpts = calcExprValues(data, expr, initExprValueMap) -// valueOpts.map { valueOpt => -// if (valueOpt.nonEmpty) { -// initExprValueMap + (expr._id -> valueOpt.get) -// } else initExprValueMap -// } -// } - // try to calculate some exprs from data and initExprValueMap, generate a new expression value map // depends on origin data and existed expr value map def genExprValueMaps(data: Option[Any], exprs: Iterable[Expr], initExprValueMap: Map[String, Any]): List[Map[String, Any]] = { - val valueMaps = calcExprsValues(data, exprs, initExprValueMap) - - valueMaps.map { valueMap => - initExprValueMap ++ valueMap - } + val (selections, nonSelections) = exprs.partition(_.isInstanceOf[SelectionExpr]) + val valueMaps = calcExprsValues(data, selections, initExprValueMap) + updateExprValueMaps(nonSelections, valueMaps) } // with exprValueMap, calculate expressions, update the expression value map // only depends on existed expr value map, only calculation, not need origin data def updateExprValueMaps(exprs: Iterable[Expr], exprValueMaps: List[Map[String, Any]]): List[Map[String, Any]] = { - exprValueMaps.flatMap { exprValueMap => - genExprValueMaps(None, exprs, exprValueMap) + exprValueMaps.map { valueMap => + exprs.foldLeft(valueMap) { (em, expr) => + expr.calculate(em) match { + case Some(v) => em + (expr._id -> v) + case _ => em + } + } } } -// private def opt2Seq(opt: Option[Any]): Seq[Option[Any]] = { -// opt match { -// case Some(v) => Seq(opt) -// case _ => Nil -// } -// } - } diff --git a/measure/src/main/scala/org/apache/griffin/measure/rule/func/DefaultFunctionDefine.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/func/DefaultFunctionDefine.scala index 341fb178f..15161c36c 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rule/func/DefaultFunctionDefine.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rule/func/DefaultFunctionDefine.scala @@ -22,9 +22,12 @@ import org.apache.griffin.measure.utils.JsonUtil class DefaultFunctionDefine extends FunctionDefine { - def json(str: String): Map[String, Any] = { + def json(strOpt: Option[_]): Map[String, Any] = { try { - JsonUtil.toAnyMap(str) + strOpt match { + case Some(str: String) => JsonUtil.toAnyMap(str) + case _ => throw new Exception("json function param should be string") + } } catch { case e: Throwable => throw e } diff --git a/measure/src/main/scala/org/apache/griffin/measure/rule/func/FunctionUtil.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/func/FunctionUtil.scala index 5451cb156..57e934d6a 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rule/func/FunctionUtil.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rule/func/FunctionUtil.scala @@ -34,7 +34,7 @@ object FunctionUtil extends Loggable { for (cls <- classes) { try { val clz: Class[_] = Class.forName(cls) - if (clz.isAssignableFrom(classOf[FunctionDefine])) { + if (classOf[FunctionDefine].isAssignableFrom(clz)) { functionDefines += (cls -> clz.newInstance.asInstanceOf[FunctionDefine]) } else { warn(s"${cls} register fails: ${cls} is not sub class of ${classOf[FunctionDefine].getCanonicalName}") @@ -46,16 +46,17 @@ object FunctionUtil extends Loggable { } def invoke(methodName: String, params: Array[Option[Any]]): Seq[Option[Any]] = { - val paramTypes = params.map { param => - try { - param match { - case Some(v) => v.getClass - case _ => classOf[UnKnown] - } - } catch { - case e: Throwable => classOf[UnKnown] - } - } +// val paramTypes = params.map { param => +// try { +// param match { +// case Some(v) => v.getClass +// case _ => classOf[UnKnown] +// } +// } catch { +// case e: Throwable => classOf[UnKnown] +// } +// } + val paramTypes = params.map(a => classOf[Option[_]]) functionDefines.values.foldLeft(Nil: Seq[Option[Any]]) { (res, funcDef) => if (res.isEmpty) { diff --git a/measure/src/test/resources/config-streaming.json b/measure/src/test/resources/config-streaming.json index 4a355486b..8e85ed2ca 100644 --- a/measure/src/test/resources/config-streaming.json +++ b/measure/src/test/resources/config-streaming.json @@ -7,12 +7,12 @@ "version": "0.8", "config": { "kafka.config": { - "bootstrap.servers": "a.b.c.d:9092", + "bootstrap.servers": "localhost:9092", "group.id": "group1", "auto.offset.reset": "smallest", "auto.commit.enable": "false", }, - "topics": "src", + "topics": "sss", "key.type": "java.lang.String", "value.type": "java.lang.String", "cache": { @@ -30,12 +30,12 @@ "version": "0.8", "config": { "kafka.config": { - "bootstrap.servers": "a.b.c.d:9092", + "bootstrap.servers": "localhost:9092", "group.id": "group1", "auto.offset.reset": "smallest", "auto.commit.enable": "false", }, - "topics": "tgt", + "topics": "ttt", "key.type": "java.lang.String", "value.type": "java.lang.String", "cache": { diff --git a/measure/src/test/resources/env-streaming.json b/measure/src/test/resources/env-streaming.json new file mode 100644 index 000000000..50f4ebf4d --- /dev/null +++ b/measure/src/test/resources/env-streaming.json @@ -0,0 +1,45 @@ +{ + "spark": { + "log.level": "INFO", + "checkpoint.dir": "hdfs:///griffin/streaming/cp", + "batch.interval": "10s", + "process.interval": "10m", + "config": {} + }, + + "persist": [ + { + "type": "hdfs", + "config": { + "path": "hdfs:///griffin/streaming/persist", + "max.persist.lines": 10000, + "max.lines.per.file": 10000 + } + }, + { + "type": "http", + "config": { + "method": "post", + "api": "http://HOSTNAME:9200/griffin/accuracy" + } + } + ], + + "info.cache": [ + { + "type": "zk", + "config": { + "hosts": "localhost:2181", + "namespace": "griffin/infocache", + "lock.path": "lock", + "mode": "persist", + "init.clear": true, + "close.clear": false + } + } + ], + + "cleaner": { + + } +} \ No newline at end of file diff --git a/measure/src/test/scala/org/apache/griffin/measure/algo/batch/BatchProfileAlgoTest.scala b/measure/src/test/scala/org/apache/griffin/measure/algo/batch/BatchProfileAlgoTest.scala index 8e11c0a71..a0fae5616 100644 --- a/measure/src/test/scala/org/apache/griffin/measure/algo/batch/BatchProfileAlgoTest.scala +++ b/measure/src/test/scala/org/apache/griffin/measure/algo/batch/BatchProfileAlgoTest.scala @@ -1,167 +1,172 @@ -///* -//Licensed to the Apache Software Foundation (ASF) under one -//or more contributor license agreements. See the NOTICE file -//distributed with this work for additional information -//regarding copyright ownership. The ASF licenses this file -//to you under the Apache License, Version 2.0 (the -//"License"); you may not use this file except in compliance -//with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -//Unless required by applicable law or agreed to in writing, -//software distributed under the License is distributed on an -//"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -//KIND, either express or implied. See the License for the -//specific language governing permissions and limitations -//under the License. -//*/ -//package org.apache.griffin.measure.algo -// -//import java.util.Date -// -//import org.apache.griffin.measure.config.params._ -//import org.apache.griffin.measure.config.params.env._ -//import org.apache.griffin.measure.config.params.user._ -//import org.apache.griffin.measure.config.reader._ -//import org.apache.griffin.measure.config.validator._ -//import org.apache.griffin.measure.connector.{DataConnector, DataConnectorFactory} -//import org.apache.griffin.measure.log.Loggable -//import org.apache.griffin.measure.rule.expr._ -//import org.apache.griffin.measure.rule.{ExprValueUtil, RuleAnalyzer, RuleFactory} -//import org.apache.spark.rdd.RDD -//import org.apache.spark.sql.SQLContext -//import org.apache.spark.{SparkConf, SparkContext} -//import org.junit.runner.RunWith -//import org.scalatest.junit.JUnitRunner -//import org.scalatest.{BeforeAndAfter, FunSuite, Matchers} -// -//import scala.util.{Failure, Success, Try} -// -// -//@RunWith(classOf[JUnitRunner]) -//class BatchProfileAlgoTest extends FunSuite with Matchers with BeforeAndAfter with Loggable { -// -// val envFile = "src/test/resources/env.json" -// val confFile = "src/test/resources/config-profile.json" -// val envFsType = "local" -// val userFsType = "local" -// -// val args = Array(envFile, confFile) -// -// var sc: SparkContext = _ -// var sqlContext: SQLContext = _ -// -// var allParam: AllParam = _ -// -// before { -// // read param files -// val envParam = readParamFile[EnvParam](envFile, envFsType) match { -// case Success(p) => p -// case Failure(ex) => { -// error(ex.getMessage) -// sys.exit(-2) -// } -// } -// val userParam = readParamFile[UserParam](confFile, userFsType) match { -// case Success(p) => p -// case Failure(ex) => { -// error(ex.getMessage) -// sys.exit(-2) -// } -// } -// allParam = AllParam(envParam, userParam) -// -// // validate param files -// validateParams(allParam) match { -// case Failure(ex) => { -// error(ex.getMessage) -// sys.exit(-3) -// } -// case _ => { -// info("params validation pass") -// } -// } -// -// val metricName = userParam.name -// val conf = new SparkConf().setMaster("local[*]").setAppName(metricName) -// sc = new SparkContext(conf) -// sqlContext = new SQLContext(sc) -// } -// -// test("algorithm") { -// Try { -// val envParam = allParam.envParam -// val userParam = allParam.userParam -// -// // start time -// val startTime = new Date().getTime() -// -// // get spark application id -// val applicationId = sc.applicationId -// -// // rules -// val ruleFactory = RuleFactory(userParam.evaluateRuleParam) -// val rule: StatementExpr = ruleFactory.generateRule() -// val ruleAnalyzer: RuleAnalyzer = RuleAnalyzer(rule) -// -// ruleAnalyzer.constCacheExprs.foreach(println) -// ruleAnalyzer.constFinalCacheExprs.foreach(println) -// -// // global cache data -// val constExprValueMap = ExprValueUtil.genExprValueMap(None, ruleAnalyzer.constCacheExprs, Map[String, Any]()) -// val finalConstExprValueMap = ExprValueUtil.updateExprValueMap(ruleAnalyzer.constFinalCacheExprs, constExprValueMap) -// -// // data connector -// val sourceDataConnector: DataConnector = -// DataConnectorFactory.getDataConnector(sqlContext, userParam.sourceParam, -// ruleAnalyzer.sourceRuleExprs, finalConstExprValueMap -// ) match { -// case Success(cntr) => { -// if (cntr.available) cntr -// else throw new Exception("source data not available!") -// } -// case Failure(ex) => throw ex -// } -// -// // get data -// val sourceData: RDD[(Product, Map[String, Any])] = sourceDataConnector.data() match { -// case Success(dt) => dt -// case Failure(ex) => throw ex -// } -// -// // my algo -// val algo = BatchProfileAlgo(allParam) -// -// // profile algorithm -// val (profileResult, missingRdd, matchedRdd) = algo.profile(sourceData, ruleAnalyzer) -// -// println(s"match percentage: ${profileResult.matchPercentage}, match count: ${profileResult.matchCount}, total count: ${profileResult.totalCount}") -// -// matchedRdd.map(rec => algo.record2String(rec, ruleAnalyzer.sourceRuleExprs.persistExprs)).foreach(println) -// -// // end time -// val endTime = new Date().getTime -// println(s"using time: ${endTime - startTime} ms") -// } match { -// case Failure(ex) => { -// error(ex.getMessage) -// sys.exit(-4) -// } -// case _ => { -// info("calculation finished") -// } -// } -// } -// -// private def readParamFile[T <: Param](file: String, fsType: String)(implicit m : Manifest[T]): Try[T] = { -// val paramReader = ParamReaderFactory.getParamReader(file, fsType) -// paramReader.readConfig[T] -// } -// -// private def validateParams(allParam: AllParam): Try[Boolean] = { -// val allParamValidator = AllParamValidator() -// allParamValidator.validate(allParam) -// } -// -//} +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ +package org.apache.griffin.measure.algo + +import java.util.Date + +import org.apache.griffin.measure.algo.batch.BatchProfileAlgo +import org.apache.griffin.measure.config.params._ +import org.apache.griffin.measure.config.params.env._ +import org.apache.griffin.measure.config.params.user._ +import org.apache.griffin.measure.config.reader._ +import org.apache.griffin.measure.config.validator._ +import org.apache.griffin.measure.connector.{BatchDataConnector, DataConnector, DataConnectorFactory} +import org.apache.griffin.measure.log.Loggable +import org.apache.griffin.measure.rule.expr._ +import org.apache.griffin.measure.rule.{ExprValueUtil, RuleAnalyzer, RuleFactory} +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.SQLContext +import org.apache.spark.{SparkConf, SparkContext} +import org.junit.runner.RunWith +import org.scalatest.junit.JUnitRunner +import org.scalatest.{BeforeAndAfter, FunSuite, Matchers} + +import scala.util.{Failure, Success, Try} + + +@RunWith(classOf[JUnitRunner]) +class BatchProfileAlgoTest extends FunSuite with Matchers with BeforeAndAfter with Loggable { + + val envFile = "src/test/resources/env.json" + val confFile = "src/test/resources/config-profile.json" + val envFsType = "local" + val userFsType = "local" + + val args = Array(envFile, confFile) + + var sc: SparkContext = _ + var sqlContext: SQLContext = _ + + var allParam: AllParam = _ + + before { + // read param files + val envParam = readParamFile[EnvParam](envFile, envFsType) match { + case Success(p) => p + case Failure(ex) => { + error(ex.getMessage) + sys.exit(-2) + } + } + val userParam = readParamFile[UserParam](confFile, userFsType) match { + case Success(p) => p + case Failure(ex) => { + error(ex.getMessage) + sys.exit(-2) + } + } + allParam = AllParam(envParam, userParam) + + // validate param files + validateParams(allParam) match { + case Failure(ex) => { + error(ex.getMessage) + sys.exit(-3) + } + case _ => { + info("params validation pass") + } + } + + val metricName = userParam.name + val conf = new SparkConf().setMaster("local[*]").setAppName(metricName) + sc = new SparkContext(conf) + sqlContext = new SQLContext(sc) + } + + test("algorithm") { + Try { + val envParam = allParam.envParam + val userParam = allParam.userParam + + // start time + val startTime = new Date().getTime() + + // get spark application id + val applicationId = sc.applicationId + + // rules + val ruleFactory = RuleFactory(userParam.evaluateRuleParam) + val rule: StatementExpr = ruleFactory.generateRule() + val ruleAnalyzer: RuleAnalyzer = RuleAnalyzer(rule) + + ruleAnalyzer.constCacheExprs.foreach(println) + ruleAnalyzer.constFinalCacheExprs.foreach(println) + + // global cache data + val constExprValueMap = ExprValueUtil.genExprValueMaps(None, ruleAnalyzer.constCacheExprs, Map[String, Any]()) + val finalConstExprValueMap = ExprValueUtil.updateExprValueMaps(ruleAnalyzer.constFinalCacheExprs, constExprValueMap) + val finalConstMap = finalConstExprValueMap.headOption match { + case Some(m) => m + case _ => Map[String, Any]() + } + + // data connector + val sourceDataConnector: BatchDataConnector = + DataConnectorFactory.getBatchDataConnector(sqlContext, userParam.sourceParam, + ruleAnalyzer.sourceRuleExprs, finalConstMap + ) match { + case Success(cntr) => { + if (cntr.available) cntr + else throw new Exception("source data not available!") + } + case Failure(ex) => throw ex + } + + // get data + val sourceData: RDD[(Product, (Map[String, Any], Map[String, Any]))] = sourceDataConnector.data() match { + case Success(dt) => dt + case Failure(ex) => throw ex + } + + // my algo + val algo = BatchProfileAlgo(allParam) + + // profile algorithm + val (profileResult, missingRdd, matchedRdd) = algo.profile(sourceData, ruleAnalyzer) + + println(s"match percentage: ${profileResult.matchPercentage}, match count: ${profileResult.matchCount}, total count: ${profileResult.totalCount}") + + matchedRdd.map(rec => algo.record2String(rec, ruleAnalyzer.sourceRuleExprs.persistExprs)).foreach(println) + + // end time + val endTime = new Date().getTime + println(s"using time: ${endTime - startTime} ms") + } match { + case Failure(ex) => { + error(ex.getMessage) + sys.exit(-4) + } + case _ => { + info("calculation finished") + } + } + } + + private def readParamFile[T <: Param](file: String, fsType: String)(implicit m : Manifest[T]): Try[T] = { + val paramReader = ParamReaderFactory.getParamReader(file, fsType) + paramReader.readConfig[T] + } + + private def validateParams(allParam: AllParam): Try[Boolean] = { + val allParamValidator = AllParamValidator() + allParamValidator.validate(allParam) + } + +} diff --git a/measure/src/test/scala/org/apache/griffin/measure/connector/ConnectorTest.scala b/measure/src/test/scala/org/apache/griffin/measure/connector/ConnectorTest.scala index 228ec2689..2642137dc 100644 --- a/measure/src/test/scala/org/apache/griffin/measure/connector/ConnectorTest.scala +++ b/measure/src/test/scala/org/apache/griffin/measure/connector/ConnectorTest.scala @@ -18,13 +18,26 @@ under the License. */ package org.apache.griffin.measure.connector +import kafka.serializer.StringDecoder import org.apache.griffin.measure.config.params.env._ +import org.apache.griffin.measure.config.params.user.{DataConnectorParam, EvaluateRuleParam} import org.apache.griffin.measure.config.reader.ParamRawStringReader +import org.apache.griffin.measure.result.{DataInfo, TimeStampInfo} +import org.apache.griffin.measure.rule.expr.{Expr, StatementExpr} +import org.apache.griffin.measure.rule.{ExprValueUtil, RuleAnalyzer, RuleFactory, RuleParser} +import org.apache.griffin.measure.utils.TimeUtil +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.SQLContext +import org.apache.spark.streaming.dstream.InputDStream +import org.apache.spark.streaming.kafka.KafkaUtils +import org.apache.spark.streaming.{Milliseconds, StreamingContext} +import org.apache.spark.{SparkConf, SparkContext} import org.junit.runner.RunWith import org.scalatest.junit.JUnitRunner import org.scalatest.{BeforeAndAfter, FunSuite, Matchers} import scala.reflect.ClassTag +import scala.util.{Failure, Success, Try} @RunWith(classOf[JUnitRunner]) class ConnectorTest extends FunSuite with Matchers with BeforeAndAfter { @@ -54,5 +67,192 @@ class ConnectorTest extends FunSuite with Matchers with BeforeAndAfter { // } // } +// "config": { +// "kafka.config": { +// "bootstrap.servers": "localhost:9092", +// "group.id": "group1", +// "auto.offset.reset": "smallest", +// "auto.commit.enable": "false", +// }, +// "topics": "sss", +// "key.type": "java.lang.String", +// "value.type": "java.lang.String", +// "cache": { +// "type": "temp", +// "config": { +// "table.name": "source", +// "info.path": "src" +// } +// } +// } + + test("connector") { + val kafkaConfig = Map[String, String]( + ("bootstrap.servers" -> "10.149.247.156:9092"), + ("group.id" -> "test"), + ("auto.offset.reset" -> "smallest"), + ("auto.commit.enable" -> "false") + ) + + val cacheConfig = Map[String, Any]( + ("table.name" -> "source"), + ("info.path" -> "src") + ) + + val cacheParam = Map[String, Any]( + ("type" -> "temp"), + ("config" -> cacheConfig) + ) + + val config = Map[String, Any]( + ("kafka.config" -> kafkaConfig), + ("topics" -> "sss"), + ("key.type" -> "java.lang.String"), + ("value.type" -> "java.lang.String"), + ("cache" -> cacheParam) + ) + + val connectorParam = DataConnectorParam("kafka", "0.8", config) + + val conf = new SparkConf().setMaster("local[*]").setAppName("ConnectorTest") + val sc = new SparkContext(conf) + val sqlContext = new SQLContext(sc) + + val batchInterval = TimeUtil.milliseconds("2s") match { + case Some(interval) => Milliseconds(interval) + case _ => throw new Exception("invalid batch interval") + } + val ssc = new StreamingContext(sc, batchInterval) + ssc.checkpoint("/test/griffin/cp") + + val connector = DataConnectorFactory.getStreamingDataConnector(ssc, connectorParam) + + val streamingConnector = connector match { + case Success(c) => c + case _ => fail + } + + /// + + val rules = "$source.json().name = 's2' AND $source.json().age = 32" + val ep = EvaluateRuleParam(1, rules) + + val ruleFactory = RuleFactory(ep) + val rule: StatementExpr = ruleFactory.generateRule() + val ruleAnalyzer: RuleAnalyzer = RuleAnalyzer(rule) + + val ruleExprs = ruleAnalyzer.sourceRuleExprs + val constFinalExprValueMap = Map[String, Any]() + + /// + + val ds = streamingConnector.stream match { + case Success(dstream) => dstream + case Failure(ex) => throw ex + } + + ds.foreachRDD((rdd, time) => { + val ms = time.milliseconds + + val data = rdd.collect + val str = data.mkString("\n") + + println(s"${ms}: \n${str}") + + val dataInfoMap = DataInfo.cacheInfoList.map(_.defWrap).toMap + TimeStampInfo.wrap(ms) + + // parse each message + val valueMapRdd: RDD[Map[String, Any]] = rdd.flatMap { kv => + val msg = kv._2 + + val cacheExprValueMaps = ExprValueUtil.genExprValueMaps(Some(msg), ruleExprs.cacheExprs, constFinalExprValueMap) + val finalExprValueMaps = ExprValueUtil.updateExprValueMaps(ruleExprs.finalCacheExprs, cacheExprValueMaps) + + // val sf = StructField("name", DataType.fromJson("string")) + // val schema: StructType = new StructType() + + finalExprValueMaps.map { vm => + vm ++ dataInfoMap + } + } + + val cnt = valueMapRdd.count + + val valueMaps = valueMapRdd.collect() + val valuestr = valueMaps.mkString("\n") + + println(s"count: ${cnt}\n ${valuestr}") + + // generate DataFrame +// val df = genDataFrame(valueMapRdd) +// +// // save data frame +// cacheDataConnector.saveData(df, ms) + }) + + + ssc.start() + ssc.awaitTermination() + ssc.stop(stopSparkContext=true, stopGracefully=true) + + // context stop + sc.stop + + } + + test ("rule calculation") { + val rules = "$source.json().name = 's2' AND $source.json().age = 32" +// val rules = "$source.json() = 'aaa'" + val ep = EvaluateRuleParam(1, rules) + + val ruleFactory = RuleFactory(ep) + val rule: StatementExpr = ruleFactory.generateRule() + val ruleAnalyzer: RuleAnalyzer = RuleAnalyzer(rule) + + val ruleExprs = ruleAnalyzer.sourceRuleExprs + val constFinalExprValueMap = Map[String, Any]() + + val data = List[String]( + ("""{"name": "s1", "age": 22}"""), + ("""{"name": "s2", "age": 32}"""), + ("""{"name": "s3", "age": 42}""") + ) + + def str(expr: Expr) = { + s"${expr._id}: ${expr.desc} [${expr.getClass.getSimpleName}]" + } + println("====") + ruleExprs.finalCacheExprs.foreach { expr => + println(str(expr)) + } + println("====") + ruleExprs.cacheExprs.foreach { expr => + println(str(expr)) + } + + val constExprValueMap = ExprValueUtil.genExprValueMaps(None, ruleAnalyzer.constCacheExprs, Map[String, Any]()) + val finalConstExprValueMap = ExprValueUtil.updateExprValueMaps(ruleAnalyzer.constFinalCacheExprs, constExprValueMap) + val finalConstMap = finalConstExprValueMap.headOption match { + case Some(m) => m + case _ => Map[String, Any]() + } + println("====") + println(ruleAnalyzer.constCacheExprs) + println(ruleAnalyzer.constFinalCacheExprs) + println(finalConstMap) + + println("====") + data.foreach { msg => + val cacheExprValueMaps = ExprValueUtil.genExprValueMaps(Some(msg), ruleExprs.cacheExprs, finalConstMap) + val finalExprValueMaps = ExprValueUtil.updateExprValueMaps(ruleExprs.finalCacheExprs, cacheExprValueMaps) + +// finalExprValueMaps.map { vm => +// vm ++ dataInfoMap +// } + println(finalExprValueMaps) + } + + } + } From fed988c7be633de09dbc4992350f0b7e7cdf5400 Mon Sep 17 00:00:00 2001 From: Lionel Liu Date: Wed, 12 Jul 2017 11:14:34 +0800 Subject: [PATCH 015/111] t4 --- .../griffin/measure/rule/ExprValueUtil.scala | 11 ++ .../measure/rule/SchemaValueCombineUtil.scala | 57 ++---- .../algo/batch/DataFrameSaveTest.scala | 171 ++++++++++++++++++ .../measure/connector/ConnectorTest.scala | 21 ++- 4 files changed, 208 insertions(+), 52 deletions(-) create mode 100644 measure/src/test/scala/org/apache/griffin/measure/algo/batch/DataFrameSaveTest.scala diff --git a/measure/src/main/scala/org/apache/griffin/measure/rule/ExprValueUtil.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/ExprValueUtil.scala index 8fdd81e25..f92d0ddd7 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rule/ExprValueUtil.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rule/ExprValueUtil.scala @@ -120,6 +120,17 @@ object ExprValueUtil { } } } +// case selector: FilterSelectExpr => { +// val field = selector.field +// pathDatas.flatMap { pathData => +// val (path, data) = pathData +// data match { +// case Some(row: Row) => { +// ; +// } +// } +// } +// } case _ => { (expr.desc :: Nil, expr.calculate(existExprValueMap)) :: Nil } diff --git a/measure/src/main/scala/org/apache/griffin/measure/rule/SchemaValueCombineUtil.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/SchemaValueCombineUtil.scala index ae12ba3a0..ed3b3fc21 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rule/SchemaValueCombineUtil.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rule/SchemaValueCombineUtil.scala @@ -24,18 +24,6 @@ object SchemaValueCombineUtil { def cartesian[T](valuesMap: Map[String, List[(List[String], T)]]): List[Map[String, T]] = { val fieldsList: List[(String, List[(List[String], T)])] = valuesMap.toList - // wrong algorithm: assume the lists have same size -// val minSize = fieldsList.map(_._2.size).min -// val idxes = (0 to (minSize - 1)).toList -// idxes.map { idx => -// fieldsList.foldLeft(Map[String, T]()) { (map, pair) => -// val (key, value) = pair -// map + (key -> value(idx)._2) -// } -// } - - // following is correct algorithm - // List[key, List[(path, value)]] to List[(path, (key, value))] val valueList: List[(List[String], (String, T))] = fieldsList.flatMap { fields => val (key, list) = fields @@ -52,36 +40,16 @@ object SchemaValueCombineUtil { val valueMapList: List[Map[String, _]] = TreeUtil.mergeDatasIntoMap(root, Nil) // 3. simple change - valueMapList.map { mp => + val result = valueMapList.map { mp => mp.map { kv => val (k, v) = kv (k, v.asInstanceOf[T]) } } - } + result -// def cartesianTest[T](valuesMap: Map[String, List[(List[String], T)]]): Unit = { -// val fieldsList: List[(String, List[(List[String], T)])] = valuesMap.toList -// -// // List[key, List[(path, value)]] to List[(path, (key, value))] -// val valueList: List[(List[String], (String, T))] = fieldsList.flatMap { fields => -// val (key, list) = fields -// list.map { pv => -// val (path, value) = pv -// (path, (key, value)) -// } -// } -// -// // 1. generate tree from value list, and return root node -// val root = TreeUtil.genRootTree(valueList) -// -// // 2. deep first visit tree from root, merge datas into value map list -// val valueMapList: List[Map[String, _]] = TreeUtil.mergeDatasIntoMap(root, Nil) -// -// valueMapList.foreach(println) -// -// } + } case class TreeNode(key: String, var datas: List[(String, _)]) { @@ -94,7 +62,6 @@ object SchemaValueCombineUtil { private def genTree(path: List[String], datas: List[(String, _)]): Option[TreeNode] = { path match { case Nil => None -// case head :: Nil => Some(TreeNode(datas)) case head :: tail => { genTree(tail, datas) match { case Some(child) => { @@ -161,12 +128,17 @@ object SchemaValueCombineUtil { def mergeDatasIntoMap(root: TreeNode, mapDatas: List[Map[String, _]]): List[Map[String, _]] = { val childrenKeysMapDatas = root.children.foldLeft(Map[List[String], List[Map[String, _]]]()) { (keysMap, child) => val childMdts = mergeDatasIntoMap(child, List[Map[String, _]]()) - val keys = keysList(childMdts) - val afterList = keysMap.get(keys) match { - case Some(list) => add(list, childMdts) - case _ => childMdts + childMdts match { + case Nil => keysMap + case _ => { + val keys = keysList(childMdts) + val afterList = keysMap.get(keys) match { + case Some(list) => add(list, childMdts) + case _ => childMdts + } + keysMap + (keys -> afterList) + } } - keysMap + (keys -> afterList) } val childrenMergeMaps = childrenKeysMapDatas.values.foldLeft(List[Map[String, _]]()) { (originList, list) => originList match { @@ -174,7 +146,8 @@ object SchemaValueCombineUtil { case _ => multiply(originList, list) } } - mergeNodeChildrenDatasIntoMap(root, childrenMergeMaps) + val result = mergeNodeChildrenDatasIntoMap(root, childrenMergeMaps) + result } private def mergeNodeChildrenDatasIntoMap(node: TreeNode, mapDatas: List[Map[String, _]]): List[Map[String, _]] = { diff --git a/measure/src/test/scala/org/apache/griffin/measure/algo/batch/DataFrameSaveTest.scala b/measure/src/test/scala/org/apache/griffin/measure/algo/batch/DataFrameSaveTest.scala new file mode 100644 index 000000000..a597760a9 --- /dev/null +++ b/measure/src/test/scala/org/apache/griffin/measure/algo/batch/DataFrameSaveTest.scala @@ -0,0 +1,171 @@ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ +package org.apache.griffin.measure.algo.batch + +import java.util.Date + +import org.apache.griffin.measure.config.params._ +import org.apache.griffin.measure.config.params.env._ +import org.apache.griffin.measure.config.params.user._ +import org.apache.griffin.measure.config.reader._ +import org.apache.griffin.measure.config.validator._ +import org.apache.griffin.measure.connector.{BatchDataConnector, DataConnectorFactory} +import org.apache.griffin.measure.log.Loggable +import org.apache.griffin.measure.rule.expr._ +import org.apache.griffin.measure.rule.{ExprValueUtil, RuleAnalyzer, RuleFactory} +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.SQLContext +import org.apache.spark.{SparkConf, SparkContext} +import org.junit.runner.RunWith +import org.scalatest.junit.JUnitRunner +import org.scalatest.{BeforeAndAfter, FunSuite, Matchers} + +import scala.util.{Failure, Success, Try} + + +@RunWith(classOf[JUnitRunner]) +class DataFrameSaveTest extends FunSuite with Matchers with BeforeAndAfter with Loggable { + + val envFile = "src/test/resources/env.json" + val confFile = "src/test/resources/config-profile.json" + val envFsType = "local" + val userFsType = "local" + + val args = Array(envFile, confFile) + + var sc: SparkContext = _ + var sqlContext: SQLContext = _ + + var allParam: AllParam = _ + + before { + // read param files + val envParam = readParamFile[EnvParam](envFile, envFsType) match { + case Success(p) => p + case Failure(ex) => { + error(ex.getMessage) + sys.exit(-2) + } + } + val userParam = readParamFile[UserParam](confFile, userFsType) match { + case Success(p) => p + case Failure(ex) => { + error(ex.getMessage) + sys.exit(-2) + } + } + allParam = AllParam(envParam, userParam) + + // validate param files + validateParams(allParam) match { + case Failure(ex) => { + error(ex.getMessage) + sys.exit(-3) + } + case _ => { + info("params validation pass") + } + } + + val metricName = userParam.name + val conf = new SparkConf().setMaster("local[*]").setAppName(metricName) + sc = new SparkContext(conf) + sqlContext = new SQLContext(sc) + } + + test("algorithm") { + Try { + val envParam = allParam.envParam + val userParam = allParam.userParam + + // start time + val startTime = new Date().getTime() + + // get spark application id + val applicationId = sc.applicationId + + // rules + val ruleFactory = RuleFactory(userParam.evaluateRuleParam) + val rule: StatementExpr = ruleFactory.generateRule() + val ruleAnalyzer: RuleAnalyzer = RuleAnalyzer(rule) + + ruleAnalyzer.constCacheExprs.foreach(println) + ruleAnalyzer.constFinalCacheExprs.foreach(println) + + // global cache data + val constExprValueMap = ExprValueUtil.genExprValueMaps(None, ruleAnalyzer.constCacheExprs, Map[String, Any]()) + val finalConstExprValueMap = ExprValueUtil.updateExprValueMaps(ruleAnalyzer.constFinalCacheExprs, constExprValueMap) + val finalConstMap = finalConstExprValueMap.headOption match { + case Some(m) => m + case _ => Map[String, Any]() + } + + // data connector + val sourceDataConnector: BatchDataConnector = + DataConnectorFactory.getBatchDataConnector(sqlContext, userParam.sourceParam, + ruleAnalyzer.sourceRuleExprs, finalConstMap + ) match { + case Success(cntr) => { + if (cntr.available) cntr + else throw new Exception("source data not available!") + } + case Failure(ex) => throw ex + } + + // get data + val sourceData: RDD[(Product, (Map[String, Any], Map[String, Any]))] = sourceDataConnector.data() match { + case Success(dt) => dt + case Failure(ex) => throw ex + } + + // my algo + val algo = BatchProfileAlgo(allParam) + + // profile algorithm + val (profileResult, missingRdd, matchedRdd) = algo.profile(sourceData, ruleAnalyzer) + + println(s"match percentage: ${profileResult.matchPercentage}, match count: ${profileResult.matchCount}, total count: ${profileResult.totalCount}") + + matchedRdd.map(rec => algo.record2String(rec, ruleAnalyzer.sourceRuleExprs.persistExprs)).foreach(println) + + // end time + val endTime = new Date().getTime + println(s"using time: ${endTime - startTime} ms") + } match { + case Failure(ex) => { + error(ex.getMessage) + sys.exit(-4) + } + case _ => { + info("calculation finished") + } + } + } + + private def readParamFile[T <: Param](file: String, fsType: String)(implicit m : Manifest[T]): Try[T] = { + val paramReader = ParamReaderFactory.getParamReader(file, fsType) + paramReader.readConfig[T] + } + + private def validateParams(allParam: AllParam): Try[Boolean] = { + val allParamValidator = AllParamValidator() + allParamValidator.validate(allParam) + } + +} diff --git a/measure/src/test/scala/org/apache/griffin/measure/connector/ConnectorTest.scala b/measure/src/test/scala/org/apache/griffin/measure/connector/ConnectorTest.scala index 2642137dc..19f37405a 100644 --- a/measure/src/test/scala/org/apache/griffin/measure/connector/ConnectorTest.scala +++ b/measure/src/test/scala/org/apache/griffin/measure/connector/ConnectorTest.scala @@ -201,8 +201,9 @@ class ConnectorTest extends FunSuite with Matchers with BeforeAndAfter { } test ("rule calculation") { - val rules = "$source.json().name = 's2' AND $source.json().age = 32" -// val rules = "$source.json() = 'aaa'" +// val rules = "$source.json().name = 's2' and $source.json().age[*] = 32" + val rules = "$source.json().items[*] = 202 AND $source.json().age[*] = 32 AND $source.json().df['a' = 1].b = 3" +// val rules = "$source.json().df[0].a = 1" val ep = EvaluateRuleParam(1, rules) val ruleFactory = RuleFactory(ep) @@ -213,9 +214,9 @@ class ConnectorTest extends FunSuite with Matchers with BeforeAndAfter { val constFinalExprValueMap = Map[String, Any]() val data = List[String]( - ("""{"name": "s1", "age": 22}"""), - ("""{"name": "s2", "age": 32}"""), - ("""{"name": "s3", "age": 42}""") + ("""{"name": "s1", "age": [22, 23], "items": [102, 104, 106], "df": [{"a": 1, "b": 3}, {"b": 2}]}"""), + ("""{"name": "s2", "age": [32, 33], "items": [202, 204, 206], "df": [{"a": 1, "b": 4}, {"b": 2}]}"""), + ("""{"name": "s3", "age": [42, 43], "items": [302, 304, 306], "df": [{"a": 1, "b": 5}, {"b": 2}]}""") ) def str(expr: Expr) = { @@ -242,16 +243,16 @@ class ConnectorTest extends FunSuite with Matchers with BeforeAndAfter { println(finalConstMap) println("====") - data.foreach { msg => + val valueMaps = data.flatMap { msg => val cacheExprValueMaps = ExprValueUtil.genExprValueMaps(Some(msg), ruleExprs.cacheExprs, finalConstMap) val finalExprValueMaps = ExprValueUtil.updateExprValueMaps(ruleExprs.finalCacheExprs, cacheExprValueMaps) -// finalExprValueMaps.map { vm => -// vm ++ dataInfoMap -// } - println(finalExprValueMaps) + finalExprValueMaps } + valueMaps.foreach(println) + println(valueMaps.size) + } } From 121d0d963e6ff3ae43bb429878f42f6202bdde65 Mon Sep 17 00:00:00 2001 From: Lionel Liu Date: Wed, 12 Jul 2017 14:41:07 +0800 Subject: [PATCH 016/111] t5 --- .../griffin/measure/rule/ExprValueUtil.scala | 94 ++++++++++++++----- .../measure/connector/ConnectorTest.scala | 3 +- 2 files changed, 74 insertions(+), 23 deletions(-) diff --git a/measure/src/main/scala/org/apache/griffin/measure/rule/ExprValueUtil.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/ExprValueUtil.scala index f92d0ddd7..4e78bbbe5 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rule/ExprValueUtil.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rule/ExprValueUtil.scala @@ -30,6 +30,30 @@ object ExprValueUtil { path :+ step } + private def getSingleValue(data: Option[Any], desc: FieldDescOnly): Option[Any] = { + data match { + case Some(row: Row) => { + desc match { + case i: IndexDesc => try { Some(row.getAs[Any](i.index)) } catch { case _ => None } + case f: FieldDesc => try { Some(row.getAs[Any](f.field)) } catch { case _ => None } + case _ => None + } + } + case Some(d: Map[String, Any]) => { + desc match { + case f: FieldDesc => d.get(f.field) + case _ => None + } + } + case Some(d: Seq[Any]) => { + desc match { + case i: IndexDesc => if (i.index >= 0 && i.index < d.size) Some(d(i.index)) else None + case _ => None + } + } + } + } + private def calcExprValues(pathDatas: List[(List[String], Option[Any])], expr: Expr, existExprValueMap: Map[String, Any]): List[(List[String], Option[Any])] = { Try { expr match { @@ -45,20 +69,44 @@ object ExprValueUtil { case Some(row: Row) => { selector.fields.flatMap { field => field match { - case i: IndexDesc => Some((append(path, i.desc), Some(row.getAs[Any](i.index)))) - case f: FieldDesc => Some((append(path, f.desc), Some(row.getAs[Any](f.field)))) - case _ => None + case (_: IndexDesc) | (_: FieldDesc) => { + getSingleValue(data, field).map { v => (append(path, field.desc), Some(v)) } + } + case a: AllFieldsDesc => { + (0 until row.size).flatMap { i => + getSingleValue(data, IndexDesc(i.toString)).map { v => + (append(path, s"${a.desc}_${i}"), Some(v)) + } + }.toList + } + case r: FieldRangeDesc => { + (r.startField, r.endField) match { + case (si: IndexDesc, ei: IndexDesc) => { + (si.index to ei.index).flatMap { i => + (append(path, s"${r.desc}_${i}"), getSingleValue(data, IndexDesc(i.toString))) + getSingleValue(data, IndexDesc(i.toString)).map { v => + (append(path, s"${r.desc}_${i}"), Some(v)) + } + }.toList + } + case _ => Nil + } + } + case _ => Nil } } } case Some(d: Map[String, Any]) => { selector.fields.flatMap { field => field match { - case f: FieldDesc => Some((append(path, f.desc), d.get(f.field))) + case (_: IndexDesc) | (_: FieldDesc) => { + getSingleValue(data, field).map { v => (append(path, field.desc), Some(v)) } + } case a: AllFieldsDesc => { - d.map { kv => - val (k, v) = kv - (append(path, s"${a.desc}_${k}"), Some(v)) + d.keySet.flatMap { k => + getSingleValue(data, FieldDesc(k)).map { v => + (append(path, s"${a.desc}_${k}"), Some(v)) + } } } case _ => None @@ -68,29 +116,28 @@ object ExprValueUtil { case Some(d: Seq[Any]) => { selector.fields.flatMap { field => field match { - case i: IndexDesc => { - if (i.index >= 0 && i.index < d.size) { - Some((append(path, i.desc), Some(d(i.index)))) - } else None + case (_: IndexDesc) | (_: FieldDesc) => { + getSingleValue(data, field).map { v => (append(path, field.desc), Some(v)) } } case a: AllFieldsDesc => { - val dt = d.zipWithIndex - dt.map { kv => - val (v, i) = kv - (append(path, s"${a.desc}_${i}"), Some(v)) - } + (0 until d.size).flatMap { i => + (append(path, s"${a.desc}_${i}"), getSingleValue(data, IndexDesc(i.toString))) + getSingleValue(data, IndexDesc(i.toString)).map { v => + (append(path, s"${a.desc}_${i}"), Some(v)) + } + }.toList } case r: FieldRangeDesc => { (r.startField, r.endField) match { case (si: IndexDesc, ei: IndexDesc) => { - if (si.index >= 0 && ei.index < d.size && si.index <= ei.index) { - val dt = d.zipWithIndex - dt.filter(kv => (kv._2 >= si.index && kv._2 <= ei.index)).map { kv => - val (v, i) = kv + (si.index to ei.index).flatMap { i => + (append(path, s"${r.desc}_${i}"), getSingleValue(data, IndexDesc(i.toString))) + getSingleValue(data, IndexDesc(i.toString)).map { v => (append(path, s"${r.desc}_${i}"), Some(v)) } - } else None + }.toList } + case _ => None } } case _ => None @@ -121,11 +168,14 @@ object ExprValueUtil { } } // case selector: FilterSelectExpr => { -// val field = selector.field // pathDatas.flatMap { pathData => // val (path, data) = pathData // data match { // case Some(row: Row) => { +// val f = selector.field +// row.getAs[Any](f.field) +// } +// case Some(d: Map[String, Any]) => { // ; // } // } diff --git a/measure/src/test/scala/org/apache/griffin/measure/connector/ConnectorTest.scala b/measure/src/test/scala/org/apache/griffin/measure/connector/ConnectorTest.scala index 19f37405a..8a782d4ae 100644 --- a/measure/src/test/scala/org/apache/griffin/measure/connector/ConnectorTest.scala +++ b/measure/src/test/scala/org/apache/griffin/measure/connector/ConnectorTest.scala @@ -202,7 +202,8 @@ class ConnectorTest extends FunSuite with Matchers with BeforeAndAfter { test ("rule calculation") { // val rules = "$source.json().name = 's2' and $source.json().age[*] = 32" - val rules = "$source.json().items[*] = 202 AND $source.json().age[*] = 32 AND $source.json().df['a' = 1].b = 3" +// val rules = "$source.json().items[*] = 202 AND $source.json().age[*] = 32 AND $source.json().df[*].a = 1" + val rules = "$source.json().items[*] = 202 AND $source.json().age[*] = 32 AND $source.json().df['a' = 1].b = 4" // val rules = "$source.json().df[0].a = 1" val ep = EvaluateRuleParam(1, rules) From 2466f6d056f392b3b5bd7d5d166eed933bf0518a Mon Sep 17 00:00:00 2001 From: Lionel Liu Date: Thu, 13 Jul 2017 13:27:12 +0800 Subject: [PATCH 017/111] t6 --- .../griffin/measure/rule/ExprValueUtil.scala | 72 +++++++++++++--- .../measure/rule/expr/LiteralExpr.scala | 4 + .../measure/rule/expr/LogicalExpr.scala | 2 +- .../measure/rule/expr/SelectExpr.scala | 15 ++++ .../measure/connector/ConnectorTest.scala | 54 ------------ .../measure/rule/ExprValueUtilTest.scala | 86 +++++++++++++++++++ 6 files changed, 164 insertions(+), 69 deletions(-) create mode 100644 measure/src/test/scala/org/apache/griffin/measure/rule/ExprValueUtilTest.scala diff --git a/measure/src/main/scala/org/apache/griffin/measure/rule/ExprValueUtil.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/ExprValueUtil.scala index 4e78bbbe5..83174edad 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rule/ExprValueUtil.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rule/ExprValueUtil.scala @@ -30,6 +30,10 @@ object ExprValueUtil { path :+ step } + private def value2Map(key: String, value: Option[Any]): Map[String, Any] = { + value.flatMap(v => Some((key -> v))).toMap + } + private def getSingleValue(data: Option[Any], desc: FieldDescOnly): Option[Any] = { data match { case Some(row: Row) => { @@ -167,20 +171,50 @@ object ExprValueUtil { } } } -// case selector: FilterSelectExpr => { -// pathDatas.flatMap { pathData => -// val (path, data) = pathData -// data match { -// case Some(row: Row) => { -// val f = selector.field -// row.getAs[Any](f.field) -// } -// case Some(d: Map[String, Any]) => { -// ; -// } -// } -// } -// } + case selector: FilterSelectExpr => { // fileter means select the items fit the condition + pathDatas.flatMap { pathData => + val (path, data) = pathData + data match { + case Some(row: Row) => { + // right value could not be selection + val rmap = value2Map(selector.value._id, selector.value.calculate(existExprValueMap)) + (0 until row.size).flatMap { i => + val dt = getSingleValue(data, IndexDesc(i.toString)) + val lmap = value2Map(selector.fieldKey, getSingleValue(dt, selector.field)) + val partValueMap = lmap ++ rmap + selector.calculate(partValueMap) match { + case Some(true) => Some((append(path, s"${selector.desc}_${i}"), dt)) + case _ => None + } + } + } + case Some(d: Map[String, Any]) => { + val rmap = value2Map(selector.value._id, selector.value.calculate(existExprValueMap)) + d.keySet.flatMap { k => + val dt = getSingleValue(data, FieldDesc(k)) + val lmap = value2Map(selector.fieldKey, getSingleValue(dt, selector.field)) + val partValueMap = lmap ++ rmap + selector.calculate(partValueMap) match { + case Some(true) => Some((append(path, s"${selector.desc}_${k}"), dt)) + case _ => None + } + } + } + case Some(d: Seq[Any]) => { + val rmap = value2Map(selector.value._id, selector.value.calculate(existExprValueMap)) + (0 until d.size).flatMap { i => + val dt = getSingleValue(data, IndexDesc(i.toString)) + val lmap = value2Map(selector.fieldKey, getSingleValue(dt, selector.field)) + val partValueMap = lmap ++ rmap + selector.calculate(partValueMap) match { + case Some(true) => Some((append(path, s"${selector.desc}_${i}"), dt)) + case _ => None + } + } + } + } + } + } case _ => { (expr.desc :: Nil, expr.calculate(existExprValueMap)) :: Nil } @@ -201,6 +235,16 @@ object ExprValueUtil { }) }.toMap SchemaValueCombineUtil.cartesian(selectionValues) +// val exprValues: Map[String, List[(List[String], Any)]] = +// exprs.foldLeft(Map[String, List[(List[String], Any)]]()) { (existExprValues, expr) => +// existExprValues + (expr._id -> calcExprValues((Nil, data) :: Nil, expr, existExprValueMap).flatMap { pair => +// pair._2 match { +// case Some(v) => Some((pair._1, v)) +// case _ => None +// } +// }) +// } +// SchemaValueCombineUtil.cartesian(exprValues) } // try to calculate some exprs from data and initExprValueMap, generate a new expression value map diff --git a/measure/src/main/scala/org/apache/griffin/measure/rule/expr/LiteralExpr.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/expr/LiteralExpr.scala index 5c78e03f7..acf158918 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rule/expr/LiteralExpr.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rule/expr/LiteralExpr.scala @@ -29,6 +29,10 @@ trait LiteralExpr extends Expr { val dataSources: Set[String] = Set.empty[String] } +case class LiteralValueExpr(value: Option[Any]) extends LiteralExpr { + val desc: String = value.getOrElse("").toString +} + case class LiteralStringExpr(expr: String) extends LiteralExpr { val value: Option[String] = Some(expr) val desc: String = s"'${value.getOrElse("")}'" diff --git a/measure/src/main/scala/org/apache/griffin/measure/rule/expr/LogicalExpr.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/expr/LogicalExpr.scala index a95e394a5..dd061d723 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rule/expr/LogicalExpr.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rule/expr/LogicalExpr.scala @@ -37,7 +37,7 @@ case class LogicalSimpleExpr(expr: MathExpr) extends LogicalExpr { case class LogicalCompareExpr(left: MathExpr, compare: String, right: MathExpr) extends LogicalExpr { private val (eqOpr, neqOpr, btOpr, bteOpr, ltOpr, lteOpr) = ("""==?""".r, """!==?""".r, ">", ">=", "<", "<=") - def calculateOnly(values: Map[String, Any]): Option[Any] = { + def calculateOnly(values: Map[String, Any]): Option[Boolean] = { val (lv, rv) = (left.calculate(values), right.calculate(values)) compare match { case this.eqOpr() => lv === rv diff --git a/measure/src/main/scala/org/apache/griffin/measure/rule/expr/SelectExpr.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/expr/SelectExpr.scala index 760c63f6b..5b7f1b00b 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rule/expr/SelectExpr.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rule/expr/SelectExpr.scala @@ -19,6 +19,7 @@ under the License. package org.apache.griffin.measure.rule.expr import org.apache.spark.sql.types.DataType +import org.apache.griffin.measure.rule.CalculationUtil._ trait SelectExpr extends Expr { def calculateOnly(values: Map[String, Any]): Option[Any] = None @@ -43,6 +44,20 @@ case class FilterSelectExpr(field: FieldDesc, compare: String, value: MathExpr) override def getSubCacheExprs(ds: String): Iterable[Expr] = value.getCacheExprs(ds) override def getSubFinalCacheExprs(ds: String): Iterable[Expr] = value.getFinalCacheExprs(ds) override def getSubPersistExprs(ds: String): Iterable[Expr] = value.getPersistExprs(ds) + private val (eqOpr, neqOpr, btOpr, bteOpr, ltOpr, lteOpr) = ("""==?""".r, """!==?""".r, ">", ">=", "<", "<=") + override def calculateOnly(values: Map[String, Any]): Option[Any] = { + val (lv, rv) = (values.get(fieldKey), value.calculate(values)) + compare match { + case this.eqOpr() => lv === rv + case this.neqOpr() => lv =!= rv + case this.btOpr => lv > rv + case this.bteOpr => lv >= rv + case this.ltOpr => lv < rv + case this.lteOpr => lv <= rv + case _ => None + } + } + def fieldKey: String = s"__${field.field}" } // -- selection -- diff --git a/measure/src/test/scala/org/apache/griffin/measure/connector/ConnectorTest.scala b/measure/src/test/scala/org/apache/griffin/measure/connector/ConnectorTest.scala index 8a782d4ae..f1f313791 100644 --- a/measure/src/test/scala/org/apache/griffin/measure/connector/ConnectorTest.scala +++ b/measure/src/test/scala/org/apache/griffin/measure/connector/ConnectorTest.scala @@ -200,61 +200,7 @@ class ConnectorTest extends FunSuite with Matchers with BeforeAndAfter { } - test ("rule calculation") { -// val rules = "$source.json().name = 's2' and $source.json().age[*] = 32" -// val rules = "$source.json().items[*] = 202 AND $source.json().age[*] = 32 AND $source.json().df[*].a = 1" - val rules = "$source.json().items[*] = 202 AND $source.json().age[*] = 32 AND $source.json().df['a' = 1].b = 4" -// val rules = "$source.json().df[0].a = 1" - val ep = EvaluateRuleParam(1, rules) - - val ruleFactory = RuleFactory(ep) - val rule: StatementExpr = ruleFactory.generateRule() - val ruleAnalyzer: RuleAnalyzer = RuleAnalyzer(rule) - val ruleExprs = ruleAnalyzer.sourceRuleExprs - val constFinalExprValueMap = Map[String, Any]() - - val data = List[String]( - ("""{"name": "s1", "age": [22, 23], "items": [102, 104, 106], "df": [{"a": 1, "b": 3}, {"b": 2}]}"""), - ("""{"name": "s2", "age": [32, 33], "items": [202, 204, 206], "df": [{"a": 1, "b": 4}, {"b": 2}]}"""), - ("""{"name": "s3", "age": [42, 43], "items": [302, 304, 306], "df": [{"a": 1, "b": 5}, {"b": 2}]}""") - ) - - def str(expr: Expr) = { - s"${expr._id}: ${expr.desc} [${expr.getClass.getSimpleName}]" - } - println("====") - ruleExprs.finalCacheExprs.foreach { expr => - println(str(expr)) - } - println("====") - ruleExprs.cacheExprs.foreach { expr => - println(str(expr)) - } - - val constExprValueMap = ExprValueUtil.genExprValueMaps(None, ruleAnalyzer.constCacheExprs, Map[String, Any]()) - val finalConstExprValueMap = ExprValueUtil.updateExprValueMaps(ruleAnalyzer.constFinalCacheExprs, constExprValueMap) - val finalConstMap = finalConstExprValueMap.headOption match { - case Some(m) => m - case _ => Map[String, Any]() - } - println("====") - println(ruleAnalyzer.constCacheExprs) - println(ruleAnalyzer.constFinalCacheExprs) - println(finalConstMap) - - println("====") - val valueMaps = data.flatMap { msg => - val cacheExprValueMaps = ExprValueUtil.genExprValueMaps(Some(msg), ruleExprs.cacheExprs, finalConstMap) - val finalExprValueMaps = ExprValueUtil.updateExprValueMaps(ruleExprs.finalCacheExprs, cacheExprValueMaps) - - finalExprValueMaps - } - - valueMaps.foreach(println) - println(valueMaps.size) - - } } diff --git a/measure/src/test/scala/org/apache/griffin/measure/rule/ExprValueUtilTest.scala b/measure/src/test/scala/org/apache/griffin/measure/rule/ExprValueUtilTest.scala new file mode 100644 index 000000000..dd8d4a06c --- /dev/null +++ b/measure/src/test/scala/org/apache/griffin/measure/rule/ExprValueUtilTest.scala @@ -0,0 +1,86 @@ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ +package org.apache.griffin.measure.rule + +import org.apache.griffin.measure.config.params.user.EvaluateRuleParam +import org.apache.griffin.measure.rule.expr.{Expr, StatementExpr} +import org.junit.runner.RunWith +import org.scalatest.junit.JUnitRunner +import org.scalatest.{BeforeAndAfter, FunSuite, Matchers} + +@RunWith(classOf[JUnitRunner]) +class ExprValueUtilTest extends FunSuite with BeforeAndAfter with Matchers { + + test ("rule calculation") { + // val rules = "$source.json().name = 's2' and $source.json().age[*] = 32" + // val rules = "$source.json().items[*] = 202 AND $source.json().age[*] = 32 AND $source.json().df[*].a = 1" + val rules = "$source.json().items[*] = 202 AND $source.json().age[*] = 32 AND $source.json().df['a' = 1].b = 4" + // val rules = "$source.json().df[0].a = 1" + val ep = EvaluateRuleParam(1, rules) + + val ruleFactory = RuleFactory(ep) + val rule: StatementExpr = ruleFactory.generateRule() + val ruleAnalyzer: RuleAnalyzer = RuleAnalyzer(rule) + + val ruleExprs = ruleAnalyzer.sourceRuleExprs + val constFinalExprValueMap = Map[String, Any]() + + val data = List[String]( + ("""{"name": "s1", "age": [22, 23], "items": [102, 104, 106], "df": [{"a": 1, "b": 3}, {"a": 2, "b": 4}]}"""), + ("""{"name": "s2", "age": [32, 33], "items": [202, 204, 206], "df": [{"a": 1, "b": 4}, {"a": 2, "b": 4}]}"""), + ("""{"name": "s3", "age": [42, 43], "items": [302, 304, 306], "df": [{"a": 1, "b": 5}, {"a": 2, "b": 4}]}""") + ) + + def str(expr: Expr) = { + s"${expr._id}: ${expr.desc} [${expr.getClass.getSimpleName}]" + } + println("====") + ruleExprs.finalCacheExprs.foreach { expr => + println(str(expr)) + } + println("====") + ruleExprs.cacheExprs.foreach { expr => + println(str(expr)) + } + + val constExprValueMap = ExprValueUtil.genExprValueMaps(None, ruleAnalyzer.constCacheExprs, Map[String, Any]()) + val finalConstExprValueMap = ExprValueUtil.updateExprValueMaps(ruleAnalyzer.constFinalCacheExprs, constExprValueMap) + val finalConstMap = finalConstExprValueMap.headOption match { + case Some(m) => m + case _ => Map[String, Any]() + } + println("====") + println(ruleAnalyzer.constCacheExprs) + println(ruleAnalyzer.constFinalCacheExprs) + println(finalConstMap) + + println("====") + val valueMaps = data.flatMap { msg => + val cacheExprValueMaps = ExprValueUtil.genExprValueMaps(Some(msg), ruleExprs.cacheExprs, finalConstMap) + val finalExprValueMaps = ExprValueUtil.updateExprValueMaps(ruleExprs.finalCacheExprs, cacheExprValueMaps) + + finalExprValueMaps + } + + valueMaps.foreach(println) + println(valueMaps.size) + + } + +} From 0f2cfd30905c7883f766a2e791289bbdbabb290e Mon Sep 17 00:00:00 2001 From: Lionel Liu Date: Fri, 14 Jul 2017 10:33:24 +0800 Subject: [PATCH 018/111] t7 --- .../src/main/resources/config-streaming.json | 6 +- .../griffin/measure/cache/ZKInfoCache.scala | 2 +- .../connector/CacheDataConnector.scala | 32 ++++++-- .../connector/DataConnectorFactory.scala | 6 +- .../connector/DfCacheDataConnector.scala | 78 ++++++++++++++++++ .../connector/KafkaDataConnector.scala | 49 ++++++----- .../connector/TempCacheDataConnector.scala | 81 ------------------- .../rule/DataTypeCalculationUtil.scala | 51 +++++++++--- .../src/test/resources/config-streaming.json | 4 +- .../measure/connector/ConnectorTest.scala | 73 +++++++++++++++-- 10 files changed, 246 insertions(+), 136 deletions(-) create mode 100644 measure/src/main/scala/org/apache/griffin/measure/connector/DfCacheDataConnector.scala delete mode 100644 measure/src/main/scala/org/apache/griffin/measure/connector/TempCacheDataConnector.scala diff --git a/measure/src/main/resources/config-streaming.json b/measure/src/main/resources/config-streaming.json index 4a355486b..023d63b9d 100644 --- a/measure/src/main/resources/config-streaming.json +++ b/measure/src/main/resources/config-streaming.json @@ -16,10 +16,12 @@ "key.type": "java.lang.String", "value.type": "java.lang.String", "cache": { - "type": "temp", + "type": "df", "config": { "table.name": "source", - "info.path": "src" + "info.path": "src", + "ready.time.interval": "1m", + "ready.time.delay": "1m" } } } diff --git a/measure/src/main/scala/org/apache/griffin/measure/cache/ZKInfoCache.scala b/measure/src/main/scala/org/apache/griffin/measure/cache/ZKInfoCache.scala index 72b0de280..cf613f2c9 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/cache/ZKInfoCache.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/cache/ZKInfoCache.scala @@ -60,7 +60,7 @@ case class ZKInfoCache(config: Map[String, Any], metricName: String) extends Inf private val cacheNamespace: String = if (namespace.isEmpty) metricName else namespace + separator + metricName private val builder = CuratorFrameworkFactory.builder() - .connectString("localhost:2181") + .connectString(hosts) .retryPolicy(new ExponentialBackoffRetry(1000, 3)) .namespace(cacheNamespace) private val client: CuratorFramework = builder.build diff --git a/measure/src/main/scala/org/apache/griffin/measure/connector/CacheDataConnector.scala b/measure/src/main/scala/org/apache/griffin/measure/connector/CacheDataConnector.scala index bff9c76e8..32ecb7bc1 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/connector/CacheDataConnector.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/connector/CacheDataConnector.scala @@ -19,28 +19,46 @@ under the License. package org.apache.griffin.measure.connector import org.apache.griffin.measure.cache.{InfoCacheInstance, ZKInfoCache} +import org.apache.griffin.measure.utils.TimeUtil import org.apache.spark.sql.DataFrame +import scala.util.Try + trait CacheDataConnector extends DataConnector { def saveData(df: DataFrame, ms: Long): Unit - def readData(): DataFrame + def readData(): Try[DataFrame] - val cacheTimeKey: String + val cacheInfoPath: String + val readyTimeInterval: Long + val readyTimeDelay: Long - val LastProcTime = "last.proc.time" - val CurReadyTime = "cur.ready.time" + def CacheTime = s"${cacheInfoPath}/cache.time" + def LastProcTime = s"${cacheInfoPath}/last.proc.time" + def ReadyTime = s"${cacheInfoPath}/ready.time" protected def submitCacheTime(ms: Long): Unit = { - val map = Map[String, String]() + (cacheTimeKey -> ms.toString) + val map = Map[String, String]((CacheTime -> ms.toString)) + InfoCacheInstance.cacheInfo(map) + } + + protected def submitReadyTime(ms: Long): Unit = { + if (ms % readyTimeInterval == 0) { + val map = Map[String, String]((ReadyTime -> ms.toString)) + InfoCacheInstance.cacheInfo(map) + } + } + + def submitLastProcTime(ms: Long): Unit = { + val map = Map[String, String]((LastProcTime -> ms.toString)) InfoCacheInstance.cacheInfo(map) } protected def readTimeRange(): (Long, Long) = { - val map = InfoCacheInstance.readInfo(List(LastProcTime, CurReadyTime)) + val map = InfoCacheInstance.readInfo(List(LastProcTime, ReadyTime)) val lastProcTime = getLong(map, LastProcTime) - val curReadyTime = getLong(map, CurReadyTime) + val curReadyTime = getLong(map, ReadyTime) (lastProcTime + 1, curReadyTime) } diff --git a/measure/src/main/scala/org/apache/griffin/measure/connector/DataConnectorFactory.scala b/measure/src/main/scala/org/apache/griffin/measure/connector/DataConnectorFactory.scala index c1a1caeb3..f1fb8f168 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/connector/DataConnectorFactory.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/connector/DataConnectorFactory.scala @@ -36,6 +36,8 @@ object DataConnectorFactory { val KafkaRegex = """^(?i)kafka$""".r + val TempRegex = """^(?i)temp|temporary$""".r + def getDataConnector(sqlContext: SQLContext, ssc: StreamingContext, dataConnectorParam: DataConnectorParam, @@ -96,8 +98,8 @@ object DataConnectorFactory { val config = dataConnectorParam.config Try { conType match { - case KafkaRegex() => { - TempCacheDataConnector(sqlContext, config) + case TempRegex() => { + DfCacheDataConnector(sqlContext, config) } case _ => throw new Exception("cache connector creation error!") } diff --git a/measure/src/main/scala/org/apache/griffin/measure/connector/DfCacheDataConnector.scala b/measure/src/main/scala/org/apache/griffin/measure/connector/DfCacheDataConnector.scala new file mode 100644 index 000000000..8bc4ff5a4 --- /dev/null +++ b/measure/src/main/scala/org/apache/griffin/measure/connector/DfCacheDataConnector.scala @@ -0,0 +1,78 @@ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ +package org.apache.griffin.measure.connector + +import org.apache.griffin.measure.utils.TimeUtil +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.{DataFrame, Row, SQLContext} +import org.apache.spark.storage.StorageLevel + +import scala.util.{Success, Try} + +case class DfCacheDataConnector(sqlContext: SQLContext, config: Map[String, Any] + ) extends CacheDataConnector { + + val CacheLevel = "cache.level" + val cacheLevel: String = config.getOrElse(CacheLevel, "MEMORY_ONLY").toString + + val timeStampColumn = "_tmst_" + + var initialed: Boolean = false + var dataFrame: DataFrame = _ + + val InfoPath = "info.path" + val cacheInfoPath: String = config.getOrElse(InfoPath, "path").toString + + val ReadyTimeInterval = "ready.time.interval" + val ReadyTimeDelay = "ready.time.delay" + val readyTimeInterval: Long = TimeUtil.milliseconds(config.getOrElse(ReadyTimeInterval, "1m").toString).getOrElse(60000L) + val readyTimeDelay: Long = TimeUtil.milliseconds(config.getOrElse(ReadyTimeDelay, "1m").toString).getOrElse(60000L) + + def available(): Boolean = { + true + } + + def saveData(df: DataFrame, ms: Long): Unit = { + if (!initialed) { + dataFrame = df + dataFrame.persist(StorageLevel.fromString(cacheLevel)) + initialed = true + } else { + if (!df.rdd.isEmpty) { + dataFrame.unpersist() + dataFrame = dataFrame.unionAll(df) + dataFrame.persist(StorageLevel.fromString(cacheLevel)) + } + } + + // submit ms + submitCacheTime(ms) + submitReadyTime(ms) + } + + def readData(): Try[DataFrame] = Try { + if (initialed) { + val timeRange = readTimeRange + dataFrame.filter(s"${timeStampColumn} BETWEEN ${timeRange._1} AND ${timeRange._2}") + } else { + throw new Exception("data not cached") + } + } + +} diff --git a/measure/src/main/scala/org/apache/griffin/measure/connector/KafkaDataConnector.scala b/measure/src/main/scala/org/apache/griffin/measure/connector/KafkaDataConnector.scala index 22adb82db..6aa691afc 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/connector/KafkaDataConnector.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/connector/KafkaDataConnector.scala @@ -148,31 +148,36 @@ case class KafkaDataConnector(sqlContext: SQLContext, ssc: StreamingContext, dat } def data(): Try[RDD[(Product, (Map[String, Any], Map[String, Any]))]] = Try { - cacheDataConnector.readData.flatMap { row => - // generate cache data - val cacheExprValueMaps = ExprValueUtil.genExprValueMaps(Some(row), ruleExprs.cacheExprs, constFinalExprValueMap) - val finalExprValueMaps = ExprValueUtil.updateExprValueMaps(ruleExprs.finalCacheExprs, cacheExprValueMaps) - - // data info - val dataInfoMap: Map[String, Any] = DataInfo.cacheInfoList.map { info => - try { - (info.key -> row.getAs[info.T](info.key)) - } catch { - case e: Throwable => info.defWrap - } - }.toMap - - finalExprValueMaps.flatMap { finalExprValueMap => - val groupbyData: Seq[AnyRef] = ruleExprs.groupbyExprs.flatMap { expr => - expr.calculate(finalExprValueMap) match { - case Some(v) => Some(v.asInstanceOf[AnyRef]) - case _ => None + cacheDataConnector.readData match { + case Success(df) => { + df.flatMap { row => + // generate cache data + val cacheExprValueMaps = ExprValueUtil.genExprValueMaps(Some(row), ruleExprs.cacheExprs, constFinalExprValueMap) + val finalExprValueMaps = ExprValueUtil.updateExprValueMaps(ruleExprs.finalCacheExprs, cacheExprValueMaps) + + // data info + val dataInfoMap: Map[String, Any] = DataInfo.cacheInfoList.map { info => + try { + (info.key -> row.getAs[info.T](info.key)) + } catch { + case e: Throwable => info.defWrap + } + }.toMap + + finalExprValueMaps.flatMap { finalExprValueMap => + val groupbyData: Seq[AnyRef] = ruleExprs.groupbyExprs.flatMap { expr => + expr.calculate(finalExprValueMap) match { + case Some(v) => Some(v.asInstanceOf[AnyRef]) + case _ => None + } + } + val key = toTuple(groupbyData) + + Some((key, (finalExprValueMap, dataInfoMap))) } } - val key = toTuple(groupbyData) - - Some((key, (finalExprValueMap, dataInfoMap))) } + case Failure(ex) => throw ex } } diff --git a/measure/src/main/scala/org/apache/griffin/measure/connector/TempCacheDataConnector.scala b/measure/src/main/scala/org/apache/griffin/measure/connector/TempCacheDataConnector.scala deleted file mode 100644 index 879a1ad7d..000000000 --- a/measure/src/main/scala/org/apache/griffin/measure/connector/TempCacheDataConnector.scala +++ /dev/null @@ -1,81 +0,0 @@ -/* -Licensed to the Apache Software Foundation (ASF) under one -or more contributor license agreements. See the NOTICE file -distributed with this work for additional information -regarding copyright ownership. The ASF licenses this file -to you under the Apache License, Version 2.0 (the -"License"); you may not use this file except in compliance -with the License. You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, -software distributed under the License is distributed on an -"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -KIND, either express or implied. See the License for the -specific language governing permissions and limitations -under the License. -*/ -package org.apache.griffin.measure.connector - -import org.apache.spark.sql.{DataFrame, SQLContext} - -import scala.util.{Try, Success} - -case class TempCacheDataConnector(sqlContext: SQLContext, config: Map[String, Any] - ) extends CacheDataConnector { - - val TableName = "table.name" - val tableName = config.getOrElse(TableName, "").toString - val tmpTableName = s"${tableName}_tmp" - - val timeStampColumn = "_tmst_" - - var tableCreated = tableExists - - val InfoPath = "info.path" - val cacheTimeKey: String = config.getOrElse(InfoPath, "").toString - - def available(): Boolean = { - tableName.nonEmpty - } - - def saveData(df: DataFrame, ms: Long): Unit = { - if (!tableCreated) { - df.registerTempTable(tableName) - sqlContext.cacheTable(tableName) - tableCreated = true - } else { - df.registerTempTable(tmpTableName) - sqlContext.sql(s"INSERT INTO TABLE ${tableName} SELECT * FROM ${tmpTableName}") - sqlContext.dropTempTable(tmpTableName) - } - - // submit ms - submitCacheTime(ms) - } - - def readData(): DataFrame = { - val timeRange = readTimeRange - val readSql = readDataSql(timeRange) - sqlContext.sql(readSql) - } - - private def tableExists(): Boolean = { - Try { - sqlContext.tables().filter(tableExistsSql).collect.size - } match { - case Success(s) => s > 0 - case _ => false - } - } - - private def tableExistsSql(): String = { - s"tableName LIKE '${tableName}'" - } - - private def readDataSql(timeRange: (Long, Long)): String = { - s"SELECT * FROM ${tableName} WHERE `${timeStampColumn}` BETWEEN ${timeRange._1} AND ${timeRange._2}" - } - -} diff --git a/measure/src/main/scala/org/apache/griffin/measure/rule/DataTypeCalculationUtil.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/DataTypeCalculationUtil.scala index 0a3781d00..9d027ec5b 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rule/DataTypeCalculationUtil.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rule/DataTypeCalculationUtil.scala @@ -43,11 +43,11 @@ object DataTypeCalculationUtil { case v: String => StringType case v: Boolean => BooleanType case v: Long => LongType - case v: Int => LongType - case v: Short => LongType - case v: Byte => LongType + case v: Int => IntegerType + case v: Short => ShortType + case v: Byte => ByteType case v: Double => DoubleType - case v: Float => DoubleType + case v: Float => FloatType case v: Map[_, _] => MapType(getSameDataType(v.keys), getSameDataType(v.values)) case v: Iterable[_] => ArrayType(getSameDataType(v)) case _ => NullType @@ -62,26 +62,53 @@ object DataTypeCalculationUtil { if (dt1 == dt2) dt1 else { dt1 match { case NullType => dt2 - case StringType => StringType + case StringType => dt1 case DoubleType => { dt2 match { - case StringType => StringType - case DoubleType | LongType => DoubleType + case StringType => dt2 + case DoubleType | FloatType | LongType | IntegerType | ShortType | ByteType => dt1 + case _ => throw DataTypeException() + } + } + case FloatType => { + dt2 match { + case StringType | DoubleType => dt2 + case FloatType | LongType | IntegerType | ShortType | ByteType => dt1 case _ => throw DataTypeException() } } case LongType => { dt2 match { - case StringType => StringType - case DoubleType => DoubleType - case LongType => LongType + case StringType | DoubleType | FloatType => dt2 + case LongType | IntegerType | ShortType | ByteType => dt1 + case _ => throw DataTypeException() + } + } + case IntegerType => { + dt2 match { + case StringType | DoubleType | FloatType | LongType => dt2 + case IntegerType | ShortType | ByteType => dt1 + case _ => throw DataTypeException() + } + } + case ShortType => { + dt2 match { + case StringType | DoubleType | FloatType | LongType | IntegerType => dt2 + case ShortType | ByteType => dt1 + case _ => throw DataTypeException() + } + } + case ByteType => { + dt2 match { + case StringType | DoubleType | FloatType | LongType | IntegerType | ShortType => dt2 + case ByteType => dt1 case _ => throw DataTypeException() } } case BooleanType => { dt2 match { - case StringType => StringType - case BooleanType => BooleanType + case StringType => dt2 + case BooleanType => dt1 case _ => throw DataTypeException() } } diff --git a/measure/src/test/resources/config-streaming.json b/measure/src/test/resources/config-streaming.json index 8e85ed2ca..c708f3c8a 100644 --- a/measure/src/test/resources/config-streaming.json +++ b/measure/src/test/resources/config-streaming.json @@ -16,9 +16,9 @@ "key.type": "java.lang.String", "value.type": "java.lang.String", "cache": { - "type": "temp", + "type": "df", "config": { - "table.name": "source", + "cache.level": "MEMORY_AND_DISK", "info.path": "src" } } diff --git a/measure/src/test/scala/org/apache/griffin/measure/connector/ConnectorTest.scala b/measure/src/test/scala/org/apache/griffin/measure/connector/ConnectorTest.scala index f1f313791..baf10defb 100644 --- a/measure/src/test/scala/org/apache/griffin/measure/connector/ConnectorTest.scala +++ b/measure/src/test/scala/org/apache/griffin/measure/connector/ConnectorTest.scala @@ -19,15 +19,18 @@ under the License. package org.apache.griffin.measure.connector import kafka.serializer.StringDecoder +import org.apache.griffin.measure.cache.InfoCacheInstance import org.apache.griffin.measure.config.params.env._ import org.apache.griffin.measure.config.params.user.{DataConnectorParam, EvaluateRuleParam} import org.apache.griffin.measure.config.reader.ParamRawStringReader import org.apache.griffin.measure.result.{DataInfo, TimeStampInfo} import org.apache.griffin.measure.rule.expr.{Expr, StatementExpr} -import org.apache.griffin.measure.rule.{ExprValueUtil, RuleAnalyzer, RuleFactory, RuleParser} +import org.apache.griffin.measure.rule._ import org.apache.griffin.measure.utils.TimeUtil +import org.apache.griffin.measure.rule.{DataTypeCalculationUtil, ExprValueUtil, RuleExprs} import org.apache.spark.rdd.RDD -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.types.{DataType, StructField, StructType} +import org.apache.spark.sql.{DataFrame, Row, SQLContext} import org.apache.spark.streaming.dstream.InputDStream import org.apache.spark.streaming.kafka.KafkaUtils import org.apache.spark.streaming.{Milliseconds, StreamingContext} @@ -112,6 +115,23 @@ class ConnectorTest extends FunSuite with Matchers with BeforeAndAfter { ("cache" -> cacheParam) ) + val infoCacheConfig = Map[String, Any]( + ("hosts" -> "localhost:2181"), + ("namespace" -> "griffin/infocache"), + ("lock.path" -> "lock"), + ("mode" -> "persist"), + ("init.clear" -> true), + ("close.clear" -> false) + ) + val name = "ttt" + + val icp = InfoCacheParam("zk", infoCacheConfig) + val icps = icp :: Nil + + InfoCacheInstance.initInstance(icps, name) + InfoCacheInstance.init + + val connectorParam = DataConnectorParam("kafka", "0.8", config) val conf = new SparkConf().setMaster("local[*]").setAppName("ConnectorTest") @@ -132,8 +152,36 @@ class ConnectorTest extends FunSuite with Matchers with BeforeAndAfter { case _ => fail } + val cacheDataConnectorParam = connectorParam.config.get("cache") match { + case Some(map: Map[String, Any]) => DataConnectorParam(map) + case _ => throw new Exception("invalid cache parameter!") + } + val cacheDataConnector = DataConnectorFactory.getCacheDataConnector(sqlContext, cacheDataConnectorParam) match { + case Success(cntr) => cntr + case Failure(ex) => throw ex + } + /// + def genDataFrame(rdd: RDD[Map[String, Any]]): DataFrame = { + val fields = rdd.aggregate(Map[String, DataType]())( + DataTypeCalculationUtil.sequenceDataTypeMap, DataTypeCalculationUtil.combineDataTypeMap + ).toList.map(f => StructField(f._1, f._2)) + val schema = StructType(fields) + val datas: RDD[Row] = rdd.map { d => + val values = fields.map { field => + val StructField(k, dt, _, _) = field + d.get(k) match { + case Some(v) => v + case _ => null + } + } + Row(values: _*) + } + val df = sqlContext.createDataFrame(datas, schema) + df + } + val rules = "$source.json().name = 's2' AND $source.json().age = 32" val ep = EvaluateRuleParam(1, rules) @@ -181,13 +229,22 @@ class ConnectorTest extends FunSuite with Matchers with BeforeAndAfter { val valueMaps = valueMapRdd.collect() val valuestr = valueMaps.mkString("\n") - println(s"count: ${cnt}\n ${valuestr}") + println(s"count: ${cnt}\n${valuestr}") // generate DataFrame -// val df = genDataFrame(valueMapRdd) -// -// // save data frame -// cacheDataConnector.saveData(df, ms) + val df = genDataFrame(valueMapRdd) + df.show(10) + + // save data frame + cacheDataConnector.saveData(df, ms) + + // show data + cacheDataConnector.readData() match { + case Success(rdf) => rdf.show(10) + case Failure(ex) => println(s"cache data error: ${ex.getMessage}") + } + + cacheDataConnector.submitLastProcTime(ms) }) @@ -198,6 +255,8 @@ class ConnectorTest extends FunSuite with Matchers with BeforeAndAfter { // context stop sc.stop + InfoCacheInstance.close() + } From 74b540297d34c76c58f5400fb3a09916409df4e8 Mon Sep 17 00:00:00 2001 From: Lionel Liu Date: Fri, 14 Jul 2017 15:28:48 +0800 Subject: [PATCH 019/111] t8 --- .../algo/batch/BatchAccuracyAlgo.scala | 2 +- .../streaming/StreamingAccuracyAlgo.scala | 2 +- .../algo/streaming/StreamingProcess.scala | 2 +- .../connector/DataConnectorFactory.scala | 18 +-- .../connector/DfCacheDataConnector.scala | 2 + .../connector/StreamingDataConnector.scala | 3 +- .../src/test/resources/config-streaming.json | 16 +- measure/src/test/resources/env-streaming.json | 8 +- .../streaming/StreamingAccuracyAlgoTest.scala | 138 ++++++++++-------- .../measure/connector/ConnectorTest.scala | 74 +++++++++- 10 files changed, 168 insertions(+), 97 deletions(-) diff --git a/measure/src/main/scala/org/apache/griffin/measure/algo/batch/BatchAccuracyAlgo.scala b/measure/src/main/scala/org/apache/griffin/measure/algo/batch/BatchAccuracyAlgo.scala index 7cc709aeb..f63b9bdb6 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/algo/batch/BatchAccuracyAlgo.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/algo/batch/BatchAccuracyAlgo.scala @@ -168,7 +168,7 @@ case class BatchAccuracyAlgo(allParam: AllParam) extends AccuracyAlgo { case vd: Map[String, Any] => getPersistMap(vd, targetPersist) case v => v } - } + }.map(identity) s"${persistData} [${persistInfo}]" } diff --git a/measure/src/main/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgo.scala b/measure/src/main/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgo.scala index 5dd650073..ea74cb8b3 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgo.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgo.scala @@ -198,7 +198,7 @@ case class StreamingAccuracyAlgo(allParam: AllParam) extends AccuracyAlgo { case vd: Map[String, Any] => getPersistMap(vd, targetPersist) case v => v } - } + }.map(identity) s"${persistData} [${persistInfo}]" } diff --git a/measure/src/main/scala/org/apache/griffin/measure/algo/streaming/StreamingProcess.scala b/measure/src/main/scala/org/apache/griffin/measure/algo/streaming/StreamingProcess.scala index e5f2b4204..28fd07067 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/algo/streaming/StreamingProcess.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/algo/streaming/StreamingProcess.scala @@ -34,7 +34,7 @@ case class StreamingProcess(interval: Long, runnable: Runnable) { } def startup(): Unit = { - timer.schedule(timerTask, 0, interval) + timer.schedule(timerTask, interval, interval) } def shutdown(): Unit = { diff --git a/measure/src/main/scala/org/apache/griffin/measure/connector/DataConnectorFactory.scala b/measure/src/main/scala/org/apache/griffin/measure/connector/DataConnectorFactory.scala index f1fb8f168..f543a4d95 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/connector/DataConnectorFactory.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/connector/DataConnectorFactory.scala @@ -36,7 +36,7 @@ object DataConnectorFactory { val KafkaRegex = """^(?i)kafka$""".r - val TempRegex = """^(?i)temp|temporary$""".r + val DfRegex = """^(?i)df|dataframe$""".r def getDataConnector(sqlContext: SQLContext, ssc: StreamingContext, @@ -98,7 +98,7 @@ object DataConnectorFactory { val config = dataConnectorParam.config Try { conType match { - case TempRegex() => { + case DfRegex() => { DfCacheDataConnector(sqlContext, config) } case _ => throw new Exception("cache connector creation error!") @@ -111,13 +111,13 @@ object DataConnectorFactory { val ValueType = "value.type" val keyType = config.getOrElse(KeyType, "java.lang.String").toString val valueType = config.getOrElse(ValueType, "java.lang.String").toString - val KafkaConfig = "kafka.config" - val Topics = "topics" - val kafkaConfig = config.get(KafkaConfig) match { - case Some(map: Map[String, Any]) => map.mapValues(_.toString) - case _ => Map[String, String]() - } - val topics = config.getOrElse(Topics, "").toString +// val KafkaConfig = "kafka.config" +// val Topics = "topics" +// val kafkaConfig = config.get(KafkaConfig) match { +// case Some(map: Map[String, Any]) => map.mapValues(_.toString).map(identity) +// case _ => Map[String, String]() +// } +// val topics = config.getOrElse(Topics, "").toString (getClassTag(keyType), getClassTag(valueType)) match { case (ClassTag(k: Class[String]), ClassTag(v: Class[String])) => { new KafkaStreamingDataConnector(ssc, config) { diff --git a/measure/src/main/scala/org/apache/griffin/measure/connector/DfCacheDataConnector.scala b/measure/src/main/scala/org/apache/griffin/measure/connector/DfCacheDataConnector.scala index 8bc4ff5a4..abc19c326 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/connector/DfCacheDataConnector.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/connector/DfCacheDataConnector.scala @@ -69,6 +69,8 @@ case class DfCacheDataConnector(sqlContext: SQLContext, config: Map[String, Any] def readData(): Try[DataFrame] = Try { if (initialed) { val timeRange = readTimeRange + println(timeRange) + submitLastProcTime(timeRange._2) dataFrame.filter(s"${timeStampColumn} BETWEEN ${timeRange._1} AND ${timeRange._2}") } else { throw new Exception("data not cached") diff --git a/measure/src/main/scala/org/apache/griffin/measure/connector/StreamingDataConnector.scala b/measure/src/main/scala/org/apache/griffin/measure/connector/StreamingDataConnector.scala index a686703fa..765d3f8b8 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/connector/StreamingDataConnector.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/connector/StreamingDataConnector.scala @@ -18,12 +18,13 @@ under the License. */ package org.apache.griffin.measure.connector +import org.apache.griffin.measure.log.Loggable import org.apache.spark.streaming.dstream.InputDStream import scala.util.Try -trait StreamingDataConnector { +trait StreamingDataConnector extends Loggable with Serializable { type K type V diff --git a/measure/src/test/resources/config-streaming.json b/measure/src/test/resources/config-streaming.json index c708f3c8a..670939f7b 100644 --- a/measure/src/test/resources/config-streaming.json +++ b/measure/src/test/resources/config-streaming.json @@ -7,10 +7,10 @@ "version": "0.8", "config": { "kafka.config": { - "bootstrap.servers": "localhost:9092", + "bootstrap.servers": "10.149.247.156:9092", "group.id": "group1", "auto.offset.reset": "smallest", - "auto.commit.enable": "false", + "auto.commit.enable": "false" }, "topics": "sss", "key.type": "java.lang.String", @@ -30,20 +30,18 @@ "version": "0.8", "config": { "kafka.config": { - "bootstrap.servers": "localhost:9092", + "bootstrap.servers": "10.149.247.156:9092", "group.id": "group1", "auto.offset.reset": "smallest", - "auto.commit.enable": "false", + "auto.commit.enable": "false" }, "topics": "ttt", "key.type": "java.lang.String", "value.type": "java.lang.String", "cache": { - "type": "hive", - "version": 1.2, + "type": "df", "config": { - "database": "default", - "table.name": "target_table", + "cache.level": "MEMORY_AND_DISK", "info.path": "tgt" } } @@ -52,6 +50,6 @@ "evaluateRule": { "sampleRatio": 0.2, - "rules": "$source.json().seeds[*].json().url = $target.json().groups[0].attrsList['name' = 'URL'].values[0] AND $source.json().seeds[*].json().metadata.json().tracker.crawlRequestCreateTS = $target.json().groups[0].attrsList['name' = 'CRAWLMETADATA'].values[0].json().tracker.crawlRequestCreateTS WHEN $source._timestamp_ + 24h < $target._timestamp_" + "rules": "$source.name = $target.name AND $source.age = $target.age WHEN $source._tmst_ < $target._tmst_ + 2m" } } \ No newline at end of file diff --git a/measure/src/test/resources/env-streaming.json b/measure/src/test/resources/env-streaming.json index 50f4ebf4d..4d72d9b0a 100644 --- a/measure/src/test/resources/env-streaming.json +++ b/measure/src/test/resources/env-streaming.json @@ -1,9 +1,9 @@ { "spark": { - "log.level": "INFO", - "checkpoint.dir": "hdfs:///griffin/streaming/cp", - "batch.interval": "10s", - "process.interval": "10m", + "log.level": "WARN", + "checkpoint.dir": "/test/griffin/cp", + "batch.interval": "2s", + "process.interval": "10s", "config": {} }, diff --git a/measure/src/test/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgoTest.scala b/measure/src/test/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgoTest.scala index df33b75a4..7c4bf4cdf 100644 --- a/measure/src/test/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgoTest.scala +++ b/measure/src/test/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgoTest.scala @@ -48,7 +48,7 @@ import scala.util.{Failure, Success, Try} @RunWith(classOf[JUnitRunner]) class StreamingAccuracyAlgoTest extends FunSuite with Matchers with BeforeAndAfter with Loggable { - val envFile = "src/test/resources/env.json" + val envFile = "src/test/resources/env-streaming.json" val confFile = "src/test/resources/config-streaming.json" val envFsType = "local" val userFsType = "local" @@ -93,6 +93,7 @@ class StreamingAccuracyAlgoTest extends FunSuite with Matchers with BeforeAndAft val metricName = userParam.name val conf = new SparkConf().setMaster("local[*]").setAppName(metricName) sc = new SparkContext(conf) + sc.setLogLevel(envParam.sparkParam.logLevel) sqlContext = new SQLContext(sc) } @@ -109,6 +110,10 @@ class StreamingAccuracyAlgoTest extends FunSuite with Matchers with BeforeAndAft val ssc = new StreamingContext(sc, batchInterval) ssc.checkpoint(sparkParam.cpDir) +// def createContext(): StreamingContext = { +// ; +// } + // start time val startTime = new Date().getTime() @@ -145,72 +150,79 @@ class StreamingAccuracyAlgoTest extends FunSuite with Matchers with BeforeAndAft } case Failure(ex) => throw ex } - val targetDataConnector: BatchDataConnector = - DataConnectorFactory.getDataConnector(sqlContext, ssc, userParam.targetParam, - ruleAnalyzer.targetRuleExprs, finalConstMap - ) match { - case Success(cntr) => { - if (cntr.available) cntr - else throw new Exception("target data connection error!") - } - case Failure(ex) => throw ex - } +// val targetDataConnector: BatchDataConnector = +// DataConnectorFactory.getDataConnector(sqlContext, ssc, userParam.targetParam, +// ruleAnalyzer.targetRuleExprs, finalConstMap +// ) match { +// case Success(cntr) => { +// if (cntr.available) cntr +// else throw new Exception("target data connection error!") +// } +// case Failure(ex) => throw ex +// } InfoCacheInstance.initInstance(envParam.infoCacheParams, metricName) InfoCacheInstance.init - // my algo - val algo = StreamingAccuracyAlgo(allParam) - - // process thread - case class Process() extends Runnable { - val lock = InfoCacheInstance.genLock("process") - def run(): Unit = { - val locked = lock.lock(5, TimeUnit.SECONDS) - if (locked) { - try { - val st = new Date().getTime - // get data - val sourceData = sourceDataConnector.data match { - case Success(dt) => dt - case Failure(ex) => throw ex - } - val targetData = targetDataConnector.data match { - case Success(dt) => dt - case Failure(ex) => throw ex - } - - // accuracy algorithm - val (accuResult, missingRdd, matchedRdd) = algo.accuracy(sourceData, targetData, ruleAnalyzer) - - println(accuResult) - - val et = new Date().getTime - - // persist time - // persist.log(et, s"calculation using time: ${et - st} ms") - - // persist result - // persist.result(et, accuResult) - val missingRecords = missingRdd.map(algo.record2String(_, ruleAnalyzer.sourceRuleExprs.persistExprs, ruleAnalyzer.targetRuleExprs.persistExprs)) - // persist.missRecords(missingRecords) - - // val pet = new Date().getTime - // persist.log(pet, s"persist using time: ${pet - et} ms") - } finally { - lock.unlock() - } - } - } - } + // init data stream + sourceDataConnector.init() +// targetDataConnector.init() - val processInterval = TimeUtil.milliseconds(sparkParam.processInterval) match { - case Some(interval) => interval - case _ => throw new Exception("invalid batch interval") - } - val process = StreamingProcess(processInterval, Process()) - - process.startup() + // my algo +// val algo = StreamingAccuracyAlgo(allParam) +// +// // process thread +// case class Process() extends Runnable { +// val lock = InfoCacheInstance.genLock("process") +// def run(): Unit = { +// val locked = lock.lock(5, TimeUnit.SECONDS) +// if (locked) { +// try { +// val st = new Date().getTime +// // get data +// val sourceData = sourceDataConnector.data match { +// case Success(dt) => dt +// case Failure(ex) => throw ex +// } +// val targetData = targetDataConnector.data match { +// case Success(dt) => dt +// case Failure(ex) => throw ex +// } +// +// println(s"sourceData.count: ${sourceData.count}") +// println(s"targetData.count: ${targetData.count}") +// +// // accuracy algorithm +//// val (accuResult, missingRdd, matchedRdd) = algo.accuracy(sourceData, targetData, ruleAnalyzer) +// +//// println(accuResult) +// +// val et = new Date().getTime +// +// // persist time +// // persist.log(et, s"calculation using time: ${et - st} ms") +// +// // persist result +// // persist.result(et, accuResult) +//// val missingRecords = missingRdd.map(algo.record2String(_, ruleAnalyzer.sourceRuleExprs.persistExprs, ruleAnalyzer.targetRuleExprs.persistExprs)) +// // persist.missRecords(missingRecords) +// +// // val pet = new Date().getTime +// // persist.log(pet, s"persist using time: ${pet - et} ms") +// } finally { +// lock.unlock() +// } +// } +// } +// } +// +// val processInterval = TimeUtil.milliseconds(sparkParam.processInterval) match { +// case Some(interval) => interval +// case _ => throw new Exception("invalid batch interval") +// } +// val process = StreamingProcess(processInterval, Process()) +// +// process.startup() ssc.start() ssc.awaitTermination() @@ -223,7 +235,7 @@ class StreamingAccuracyAlgoTest extends FunSuite with Matchers with BeforeAndAft // persist.finish() - process.shutdown() +// process.shutdown() } private def readParamFile[T <: Param](file: String, fsType: String)(implicit m : Manifest[T]): Try[T] = { diff --git a/measure/src/test/scala/org/apache/griffin/measure/connector/ConnectorTest.scala b/measure/src/test/scala/org/apache/griffin/measure/connector/ConnectorTest.scala index baf10defb..0f3890438 100644 --- a/measure/src/test/scala/org/apache/griffin/measure/connector/ConnectorTest.scala +++ b/measure/src/test/scala/org/apache/griffin/measure/connector/ConnectorTest.scala @@ -18,7 +18,11 @@ under the License. */ package org.apache.griffin.measure.connector +import java.util.Date +import java.util.concurrent.TimeUnit + import kafka.serializer.StringDecoder +import org.apache.griffin.measure.algo.streaming.StreamingProcess import org.apache.griffin.measure.cache.InfoCacheInstance import org.apache.griffin.measure.config.params.env._ import org.apache.griffin.measure.config.params.user.{DataConnectorParam, EvaluateRuleParam} @@ -103,7 +107,7 @@ class ConnectorTest extends FunSuite with Matchers with BeforeAndAfter { ) val cacheParam = Map[String, Any]( - ("type" -> "temp"), + ("type" -> "df"), ("config" -> cacheConfig) ) @@ -136,6 +140,7 @@ class ConnectorTest extends FunSuite with Matchers with BeforeAndAfter { val conf = new SparkConf().setMaster("local[*]").setAppName("ConnectorTest") val sc = new SparkContext(conf) + sc.setLogLevel("WARN") val sqlContext = new SQLContext(sc) val batchInterval = TimeUtil.milliseconds("2s") match { @@ -229,23 +234,74 @@ class ConnectorTest extends FunSuite with Matchers with BeforeAndAfter { val valueMaps = valueMapRdd.collect() val valuestr = valueMaps.mkString("\n") - println(s"count: ${cnt}\n${valuestr}") +// println(s"count: ${cnt}\n${valuestr}") // generate DataFrame val df = genDataFrame(valueMapRdd) - df.show(10) +// df.show(10) // save data frame cacheDataConnector.saveData(df, ms) // show data - cacheDataConnector.readData() match { - case Success(rdf) => rdf.show(10) - case Failure(ex) => println(s"cache data error: ${ex.getMessage}") +// cacheDataConnector.readData() match { +// case Success(rdf) => rdf.show(10) +// case Failure(ex) => println(s"cache data error: ${ex.getMessage}") +// } +// +// cacheDataConnector.submitLastProcTime(ms) + }) + + // process thread + case class Process() extends Runnable { + val lock = InfoCacheInstance.genLock("process") + def run(): Unit = { + val locked = lock.lock(5, TimeUnit.SECONDS) + if (locked) { + try { + // show data + cacheDataConnector.readData() match { + case Success(rdf) => { + rdf.show(10) + println(s"count: ${rdf.count}") + } + case Failure(ex) => println(s"cache data error: ${ex.getMessage}") + } + +// val st = new Date().getTime + // get data +// val sourceData = sourceDataConnector.data match { +// case Success(dt) => dt +// case Failure(ex) => throw ex +// } +// val targetData = targetDataConnector.data match { +// case Success(dt) => dt +// case Failure(ex) => throw ex +// } +// +// // accuracy algorithm +// val (accuResult, missingRdd, matchedRdd) = accuracy(sourceData, targetData, ruleAnalyzer) +// +// println(accuResult) +// +// val et = new Date().getTime +// +// val missingRecords = missingRdd.map(record2String(_, ruleAnalyzer.sourceRuleExprs.persistExprs, ruleAnalyzer.targetRuleExprs.persistExprs)) + + } finally { + lock.unlock() + } + } } + } - cacheDataConnector.submitLastProcTime(ms) - }) + val processInterval = TimeUtil.milliseconds("10s") match { + case Some(interval) => interval + case _ => throw new Exception("invalid batch interval") + } + val process = StreamingProcess(processInterval, Process()) + + process.startup() ssc.start() @@ -257,6 +313,8 @@ class ConnectorTest extends FunSuite with Matchers with BeforeAndAfter { InfoCacheInstance.close() + process.shutdown() + } From 86a0425a85d64db64ec08a63593f0fc82232fb9f Mon Sep 17 00:00:00 2001 From: Lionel Liu Date: Sun, 16 Jul 2017 16:33:23 +0800 Subject: [PATCH 020/111] t9 --- .../connector/DfCacheDataConnector.scala | 6 +- .../connector/KafkaDataConnector.scala | 44 +++--- .../connector/StreamingDataConnector.scala | 2 +- .../griffin/measure/rule/RuleAnalyzer.scala | 2 +- .../src/test/resources/config-streaming.json | 2 +- .../streaming/StreamingAccuracyAlgoTest.scala | 132 +++++++++--------- 6 files changed, 92 insertions(+), 96 deletions(-) diff --git a/measure/src/main/scala/org/apache/griffin/measure/connector/DfCacheDataConnector.scala b/measure/src/main/scala/org/apache/griffin/measure/connector/DfCacheDataConnector.scala index abc19c326..0f302227f 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/connector/DfCacheDataConnector.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/connector/DfCacheDataConnector.scala @@ -18,6 +18,7 @@ under the License. */ package org.apache.griffin.measure.connector +import org.apache.griffin.measure.result.TimeStampInfo import org.apache.griffin.measure.utils.TimeUtil import org.apache.spark.rdd.RDD import org.apache.spark.sql.{DataFrame, Row, SQLContext} @@ -26,12 +27,12 @@ import org.apache.spark.storage.StorageLevel import scala.util.{Success, Try} case class DfCacheDataConnector(sqlContext: SQLContext, config: Map[String, Any] - ) extends CacheDataConnector { + ) extends CacheDataConnector { val CacheLevel = "cache.level" val cacheLevel: String = config.getOrElse(CacheLevel, "MEMORY_ONLY").toString - val timeStampColumn = "_tmst_" + val timeStampColumn = TimeStampInfo.key var initialed: Boolean = false var dataFrame: DataFrame = _ @@ -69,7 +70,6 @@ case class DfCacheDataConnector(sqlContext: SQLContext, config: Map[String, Any] def readData(): Try[DataFrame] = Try { if (initialed) { val timeRange = readTimeRange - println(timeRange) submitLastProcTime(timeRange._2) dataFrame.filter(s"${timeStampColumn} BETWEEN ${timeRange._1} AND ${timeRange._2}") } else { diff --git a/measure/src/main/scala/org/apache/griffin/measure/connector/KafkaDataConnector.scala b/measure/src/main/scala/org/apache/griffin/measure/connector/KafkaDataConnector.scala index 6aa691afc..c3687906c 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/connector/KafkaDataConnector.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/connector/KafkaDataConnector.scala @@ -18,9 +18,12 @@ under the License. */ package org.apache.griffin.measure.connector -import org.apache.griffin.measure.config.params.user.DataConnectorParam +import java.util.Date + +import org.apache.griffin.measure.config.params.user.{DataConnectorParam, EvaluateRuleParam} import org.apache.griffin.measure.result._ -import org.apache.griffin.measure.rule.{DataTypeCalculationUtil, ExprValueUtil, RuleExprs} +import org.apache.griffin.measure.rule.expr.StatementExpr +import org.apache.griffin.measure.rule._ import org.apache.griffin.measure.utils.TimeUtil import org.apache.spark.rdd.RDD import org.apache.spark.sql.types.{DataType, StructField, StructType} @@ -29,11 +32,11 @@ import org.apache.spark.sql.{DataFrame, Row, SQLContext} import scala.util.{Failure, Success, Try} -case class KafkaDataConnector(sqlContext: SQLContext, ssc: StreamingContext, dataConnectorParam: DataConnectorParam, +case class KafkaDataConnector(sqlContext: SQLContext, @transient ssc: StreamingContext, dataConnectorParam: DataConnectorParam, ruleExprs: RuleExprs, constFinalExprValueMap: Map[String, Any] ) extends BatchDataConnector { - val kafkaStreamingDataConnector = DataConnectorFactory.getStreamingDataConnector(ssc, dataConnectorParam) match { + @transient val kafkaStreamingDataConnector = DataConnectorFactory.getStreamingDataConnector(ssc, dataConnectorParam) match { case Success(cntr) => cntr case Failure(ex) => throw ex } @@ -87,14 +90,9 @@ case class KafkaDataConnector(sqlContext: SQLContext, ssc: StreamingContext, dat case Success(dstream) => dstream case Failure(ex) => throw ex } + ds.foreachRDD((rdd, time) => { val ms = time.milliseconds -// val min = TimeUtil.timeToUnit(ms, "min") -// val hour = TimeUtil.timeToUnit(ms, "hour") -// val partitions = List[(String, Any)](("hr", hour), ("min", min)) - -// val partitionPath = genPartitionHdfsPath(partitions) -// val path = s"${targetDumpDir}/${partitionPath}/${ms}" val dataInfoMap = DataInfo.cacheInfoList.map(_.defWrap).toMap + TimeStampInfo.wrap(ms) @@ -105,9 +103,6 @@ case class KafkaDataConnector(sqlContext: SQLContext, ssc: StreamingContext, dat val cacheExprValueMaps = ExprValueUtil.genExprValueMaps(Some(msg), ruleExprs.cacheExprs, constFinalExprValueMap) val finalExprValueMaps = ExprValueUtil.updateExprValueMaps(ruleExprs.finalCacheExprs, cacheExprValueMaps) -// val sf = StructField("name", DataType.fromJson("string")) -// val schema: StructType = new StructType() - finalExprValueMaps.map { vm => vm ++ dataInfoMap } @@ -152,8 +147,11 @@ case class KafkaDataConnector(sqlContext: SQLContext, ssc: StreamingContext, dat case Success(df) => { df.flatMap { row => // generate cache data - val cacheExprValueMaps = ExprValueUtil.genExprValueMaps(Some(row), ruleExprs.cacheExprs, constFinalExprValueMap) - val finalExprValueMaps = ExprValueUtil.updateExprValueMaps(ruleExprs.finalCacheExprs, cacheExprValueMaps) +// val cacheExprValueMaps = ExprValueUtil.genExprValueMaps(Some(row), ruleExprs.cacheExprs, constFinalExprValueMap) +// val finalExprValueMaps = ExprValueUtil.updateExprValueMaps(ruleExprs.finalCacheExprs, cacheExprValueMaps) + val finalExprValueMap = ruleExprs.finalCacheExprs.foldLeft(Map[String, Any]()) { (mp, expr) => + mp + (expr._id -> row.getAs[Any](expr._id)) + } // data info val dataInfoMap: Map[String, Any] = DataInfo.cacheInfoList.map { info => @@ -164,17 +162,15 @@ case class KafkaDataConnector(sqlContext: SQLContext, ssc: StreamingContext, dat } }.toMap - finalExprValueMaps.flatMap { finalExprValueMap => - val groupbyData: Seq[AnyRef] = ruleExprs.groupbyExprs.flatMap { expr => - expr.calculate(finalExprValueMap) match { - case Some(v) => Some(v.asInstanceOf[AnyRef]) - case _ => None - } + val groupbyData: Seq[AnyRef] = ruleExprs.groupbyExprs.flatMap { expr => + expr.calculate(finalExprValueMap) match { + case Some(v) => Some(v.asInstanceOf[AnyRef]) + case _ => None } - val key = toTuple(groupbyData) - - Some((key, (finalExprValueMap, dataInfoMap))) } + val key = toTuple(groupbyData) + + Some((key, (finalExprValueMap, dataInfoMap))) } } case Failure(ex) => throw ex diff --git a/measure/src/main/scala/org/apache/griffin/measure/connector/StreamingDataConnector.scala b/measure/src/main/scala/org/apache/griffin/measure/connector/StreamingDataConnector.scala index 765d3f8b8..b597f6f4b 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/connector/StreamingDataConnector.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/connector/StreamingDataConnector.scala @@ -24,7 +24,7 @@ import org.apache.spark.streaming.dstream.InputDStream import scala.util.Try -trait StreamingDataConnector extends Loggable with Serializable { +trait StreamingDataConnector extends Loggable { type K type V diff --git a/measure/src/main/scala/org/apache/griffin/measure/rule/RuleAnalyzer.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/RuleAnalyzer.scala index 9e7b6dcde..bade43e26 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rule/RuleAnalyzer.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rule/RuleAnalyzer.scala @@ -64,7 +64,7 @@ case class RuleExprs(groupbyExprs: Seq[Expr], cacheExprs: Iterable[Expr], finalCacheExprs: Iterable[Expr], persistExprs: Iterable[Expr] - ) { + ) extends Serializable { // for example: for a rule "$source.name = $target.name AND $source.age < $target.age + (3 * 4)" // in this rule, for the target data source, the targetRuleExprs looks like below // groupbyExprs: $target.name diff --git a/measure/src/test/resources/config-streaming.json b/measure/src/test/resources/config-streaming.json index 670939f7b..a7a04e768 100644 --- a/measure/src/test/resources/config-streaming.json +++ b/measure/src/test/resources/config-streaming.json @@ -50,6 +50,6 @@ "evaluateRule": { "sampleRatio": 0.2, - "rules": "$source.name = $target.name AND $source.age = $target.age WHEN $source._tmst_ < $target._tmst_ + 2m" + "rules": "$source.json().name = $target.json().name AND $source.json().age = $target.json().age WHEN $source._tmst_ < $target._tmst_ + 2m" } } \ No newline at end of file diff --git a/measure/src/test/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgoTest.scala b/measure/src/test/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgoTest.scala index 7c4bf4cdf..2e6cb72c6 100644 --- a/measure/src/test/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgoTest.scala +++ b/measure/src/test/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgoTest.scala @@ -150,79 +150,79 @@ class StreamingAccuracyAlgoTest extends FunSuite with Matchers with BeforeAndAft } case Failure(ex) => throw ex } -// val targetDataConnector: BatchDataConnector = -// DataConnectorFactory.getDataConnector(sqlContext, ssc, userParam.targetParam, -// ruleAnalyzer.targetRuleExprs, finalConstMap -// ) match { -// case Success(cntr) => { -// if (cntr.available) cntr -// else throw new Exception("target data connection error!") -// } -// case Failure(ex) => throw ex -// } + val targetDataConnector: BatchDataConnector = + DataConnectorFactory.getDataConnector(sqlContext, ssc, userParam.targetParam, + ruleAnalyzer.targetRuleExprs, finalConstMap + ) match { + case Success(cntr) => { + if (cntr.available) cntr + else throw new Exception("target data connection error!") + } + case Failure(ex) => throw ex + } InfoCacheInstance.initInstance(envParam.infoCacheParams, metricName) InfoCacheInstance.init // init data stream sourceDataConnector.init() -// targetDataConnector.init() + targetDataConnector.init() // my algo -// val algo = StreamingAccuracyAlgo(allParam) -// -// // process thread -// case class Process() extends Runnable { -// val lock = InfoCacheInstance.genLock("process") -// def run(): Unit = { -// val locked = lock.lock(5, TimeUnit.SECONDS) -// if (locked) { -// try { -// val st = new Date().getTime -// // get data -// val sourceData = sourceDataConnector.data match { -// case Success(dt) => dt -// case Failure(ex) => throw ex -// } -// val targetData = targetDataConnector.data match { -// case Success(dt) => dt -// case Failure(ex) => throw ex -// } -// -// println(s"sourceData.count: ${sourceData.count}") -// println(s"targetData.count: ${targetData.count}") -// -// // accuracy algorithm -//// val (accuResult, missingRdd, matchedRdd) = algo.accuracy(sourceData, targetData, ruleAnalyzer) -// -//// println(accuResult) -// -// val et = new Date().getTime -// -// // persist time -// // persist.log(et, s"calculation using time: ${et - st} ms") -// -// // persist result -// // persist.result(et, accuResult) -//// val missingRecords = missingRdd.map(algo.record2String(_, ruleAnalyzer.sourceRuleExprs.persistExprs, ruleAnalyzer.targetRuleExprs.persistExprs)) -// // persist.missRecords(missingRecords) -// -// // val pet = new Date().getTime -// // persist.log(pet, s"persist using time: ${pet - et} ms") -// } finally { -// lock.unlock() -// } -// } -// } -// } -// -// val processInterval = TimeUtil.milliseconds(sparkParam.processInterval) match { -// case Some(interval) => interval -// case _ => throw new Exception("invalid batch interval") -// } -// val process = StreamingProcess(processInterval, Process()) -// -// process.startup() + val algo = StreamingAccuracyAlgo(allParam) + + // process thread + case class Process() extends Runnable { + val lock = InfoCacheInstance.genLock("process") + def run(): Unit = { + val locked = lock.lock(5, TimeUnit.SECONDS) + if (locked) { + try { + val st = new Date().getTime + // get data + val sourceData = sourceDataConnector.data match { + case Success(dt) => dt + case Failure(ex) => throw ex + } + val targetData = targetDataConnector.data match { + case Success(dt) => dt + case Failure(ex) => throw ex + } + + println(s"sourceData.count: ${sourceData.count}") + println(s"targetData.count: ${targetData.count}") + + // accuracy algorithm + val (accuResult, missingRdd, matchedRdd) = algo.accuracy(sourceData, targetData, ruleAnalyzer) + println(accuResult) + + val et = new Date().getTime + println(s"process time: ${et - st} ms") + + // persist time + // persist.log(et, s"calculation using time: ${et - st} ms") + + // persist result + // persist.result(et, accuResult) +// val missingRecords = missingRdd.map(algo.record2String(_, ruleAnalyzer.sourceRuleExprs.persistExprs, ruleAnalyzer.targetRuleExprs.persistExprs)) + // persist.missRecords(missingRecords) + + // val pet = new Date().getTime + // persist.log(pet, s"persist using time: ${pet - et} ms") + } finally { + lock.unlock() + } + } + } + } + + val processInterval = TimeUtil.milliseconds(sparkParam.processInterval) match { + case Some(interval) => interval + case _ => throw new Exception("invalid batch interval") + } + val process = StreamingProcess(processInterval, Process()) + + process.startup() ssc.start() ssc.awaitTermination() @@ -235,7 +235,7 @@ class StreamingAccuracyAlgoTest extends FunSuite with Matchers with BeforeAndAft // persist.finish() -// process.shutdown() + process.shutdown() } private def readParamFile[T <: Param](file: String, fsType: String)(implicit m : Manifest[T]): Try[T] = { From 8bbf450280ce7a5d85ab56d66c925e515eabd502 Mon Sep 17 00:00:00 2001 From: Lionel Liu Date: Mon, 17 Jul 2017 13:57:39 +0800 Subject: [PATCH 021/111] t9 --- .../measure/algo/core/AccuracyCore.scala | 2 + .../streaming/StreamingAccuracyAlgo.scala | 16 +++- .../measure/cache/{ => info}/InfoCache.scala | 3 +- .../cache/{ => info}/InfoCacheFactory.scala | 2 +- .../cache/{ => info}/InfoCacheInstance.scala | 3 +- .../cache/{ => info}/ZKInfoCache.scala | 3 +- .../measure/cache/{ => lock}/CacheLock.scala | 2 +- .../cache/{ => lock}/MultiCacheLock.scala | 2 +- .../cache/{ => lock}/ZKCacheLock.scala | 2 +- .../measure/cache/result/CacheResult.scala | 29 +++++++ .../cache/result/CacheResultProcesser.scala | 73 ++++++++++++++++++ .../connector/CacheDataConnector.scala | 2 +- .../griffin/measure/rule/RuleAnalyzer.scala | 5 +- .../streaming/StreamingAccuracyAlgoTest.scala | 77 ++++++++++++++++++- .../measure/cache/InfoCacheInstanceTest.scala | 1 + .../measure/cache/ZKCacheLockTest.scala | 1 + .../measure/cache/ZKInfoCacheTest.scala | 1 + .../measure/connector/ConnectorTest.scala | 2 +- .../src/main/resources/sparkJob.properties | 2 +- 19 files changed, 210 insertions(+), 18 deletions(-) rename measure/src/main/scala/org/apache/griffin/measure/cache/{ => info}/InfoCache.scala (91%) rename measure/src/main/scala/org/apache/griffin/measure/cache/{ => info}/InfoCacheFactory.scala (96%) rename measure/src/main/scala/org/apache/griffin/measure/cache/{ => info}/InfoCacheInstance.scala (94%) rename measure/src/main/scala/org/apache/griffin/measure/cache/{ => info}/ZKInfoCache.scala (98%) rename measure/src/main/scala/org/apache/griffin/measure/cache/{ => lock}/CacheLock.scala (95%) rename measure/src/main/scala/org/apache/griffin/measure/cache/{ => lock}/MultiCacheLock.scala (96%) rename measure/src/main/scala/org/apache/griffin/measure/cache/{ => lock}/ZKCacheLock.scala (96%) create mode 100644 measure/src/main/scala/org/apache/griffin/measure/cache/result/CacheResult.scala create mode 100644 measure/src/main/scala/org/apache/griffin/measure/cache/result/CacheResultProcesser.scala diff --git a/measure/src/main/scala/org/apache/griffin/measure/algo/core/AccuracyCore.scala b/measure/src/main/scala/org/apache/griffin/measure/algo/core/AccuracyCore.scala index f3f827c0d..4ec6505c8 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/algo/core/AccuracyCore.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/algo/core/AccuracyCore.scala @@ -94,6 +94,8 @@ object AccuracyCore { } +// private def when + private def mergeExprValueMap(source: (V, T), target: (V, T)): Map[String, Any] = { source._1 ++ target._1 } diff --git a/measure/src/main/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgo.scala b/measure/src/main/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgo.scala index ea74cb8b3..0099235a4 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgo.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgo.scala @@ -23,11 +23,11 @@ import java.util.concurrent.{Executors, ThreadPoolExecutor, TimeUnit} import org.apache.griffin.measure.algo.AccuracyAlgo import org.apache.griffin.measure.algo.core.AccuracyCore -import org.apache.griffin.measure.cache.InfoCacheInstance +import org.apache.griffin.measure.cache.info.InfoCacheInstance import org.apache.griffin.measure.config.params.AllParam import org.apache.griffin.measure.connector._ import org.apache.griffin.measure.persist.{Persist, PersistFactory} -import org.apache.griffin.measure.result.AccuracyResult +import org.apache.griffin.measure.result.{AccuracyResult, TimeStampInfo} import org.apache.griffin.measure.rule.{ExprValueUtil, RuleAnalyzer, RuleFactory} import org.apache.griffin.measure.rule.expr._ import org.apache.griffin.measure.utils.TimeUtil @@ -214,4 +214,16 @@ case class StreamingAccuracyAlgo(allParam: AllParam) extends AccuracyAlgo { } } + def reorgByTimeGroup(rdd: RDD[(Product, (Map[String, Any], Map[String, Any]))] + ): RDD[(Long, (Product, (Map[String, Any], Map[String, Any])))] = { + rdd.flatMap { row => + val (key, (value, info)) = row + val b: Option[(Long, (Product, (Map[String, Any], Map[String, Any])))] = info.get(TimeStampInfo.key) match { + case Some(t: Long) => Some((t, row)) + case _ => None + } + b + } + } + } diff --git a/measure/src/main/scala/org/apache/griffin/measure/cache/InfoCache.scala b/measure/src/main/scala/org/apache/griffin/measure/cache/info/InfoCache.scala similarity index 91% rename from measure/src/main/scala/org/apache/griffin/measure/cache/InfoCache.scala rename to measure/src/main/scala/org/apache/griffin/measure/cache/info/InfoCache.scala index 3308fd6c2..f7250a4ec 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/cache/InfoCache.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/cache/info/InfoCache.scala @@ -16,8 +16,9 @@ KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License. */ -package org.apache.griffin.measure.cache +package org.apache.griffin.measure.cache.info +import org.apache.griffin.measure.cache.lock.CacheLock import org.apache.griffin.measure.log.Loggable trait InfoCache extends Loggable with Serializable { diff --git a/measure/src/main/scala/org/apache/griffin/measure/cache/InfoCacheFactory.scala b/measure/src/main/scala/org/apache/griffin/measure/cache/info/InfoCacheFactory.scala similarity index 96% rename from measure/src/main/scala/org/apache/griffin/measure/cache/InfoCacheFactory.scala rename to measure/src/main/scala/org/apache/griffin/measure/cache/info/InfoCacheFactory.scala index 82369b2a3..3c9d70a36 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/cache/InfoCacheFactory.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/cache/info/InfoCacheFactory.scala @@ -16,7 +16,7 @@ KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License. */ -package org.apache.griffin.measure.cache +package org.apache.griffin.measure.cache.info import org.apache.griffin.measure.config.params.env.InfoCacheParam diff --git a/measure/src/main/scala/org/apache/griffin/measure/cache/InfoCacheInstance.scala b/measure/src/main/scala/org/apache/griffin/measure/cache/info/InfoCacheInstance.scala similarity index 94% rename from measure/src/main/scala/org/apache/griffin/measure/cache/InfoCacheInstance.scala rename to measure/src/main/scala/org/apache/griffin/measure/cache/info/InfoCacheInstance.scala index 95c595ea2..4bb50a549 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/cache/InfoCacheInstance.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/cache/info/InfoCacheInstance.scala @@ -16,8 +16,9 @@ KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License. */ -package org.apache.griffin.measure.cache +package org.apache.griffin.measure.cache.info +import org.apache.griffin.measure.cache.lock.{CacheLock, MultiCacheLock} import org.apache.griffin.measure.config.params.env.InfoCacheParam object InfoCacheInstance extends InfoCache { diff --git a/measure/src/main/scala/org/apache/griffin/measure/cache/ZKInfoCache.scala b/measure/src/main/scala/org/apache/griffin/measure/cache/info/ZKInfoCache.scala similarity index 98% rename from measure/src/main/scala/org/apache/griffin/measure/cache/ZKInfoCache.scala rename to measure/src/main/scala/org/apache/griffin/measure/cache/info/ZKInfoCache.scala index cf613f2c9..5417b3a9c 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/cache/ZKInfoCache.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/cache/info/ZKInfoCache.scala @@ -16,12 +16,13 @@ KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License. */ -package org.apache.griffin.measure.cache +package org.apache.griffin.measure.cache.info import org.apache.curator.framework.imps.CuratorFrameworkState import org.apache.curator.framework.recipes.locks.InterProcessMutex import org.apache.curator.framework.{CuratorFramework, CuratorFrameworkFactory} import org.apache.curator.retry.ExponentialBackoffRetry +import org.apache.griffin.measure.cache.lock.ZKCacheLock import org.apache.zookeeper.CreateMode case class ZKInfoCache(config: Map[String, Any], metricName: String) extends InfoCache { diff --git a/measure/src/main/scala/org/apache/griffin/measure/cache/CacheLock.scala b/measure/src/main/scala/org/apache/griffin/measure/cache/lock/CacheLock.scala similarity index 95% rename from measure/src/main/scala/org/apache/griffin/measure/cache/CacheLock.scala rename to measure/src/main/scala/org/apache/griffin/measure/cache/lock/CacheLock.scala index e1c7047ca..24f142c5d 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/cache/CacheLock.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/cache/lock/CacheLock.scala @@ -16,7 +16,7 @@ KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License. */ -package org.apache.griffin.measure.cache +package org.apache.griffin.measure.cache.lock import java.util.concurrent.TimeUnit diff --git a/measure/src/main/scala/org/apache/griffin/measure/cache/MultiCacheLock.scala b/measure/src/main/scala/org/apache/griffin/measure/cache/lock/MultiCacheLock.scala similarity index 96% rename from measure/src/main/scala/org/apache/griffin/measure/cache/MultiCacheLock.scala rename to measure/src/main/scala/org/apache/griffin/measure/cache/lock/MultiCacheLock.scala index bc7d626a3..7b835f4b1 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/cache/MultiCacheLock.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/cache/lock/MultiCacheLock.scala @@ -16,7 +16,7 @@ KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License. */ -package org.apache.griffin.measure.cache +package org.apache.griffin.measure.cache.lock import java.util.concurrent.TimeUnit diff --git a/measure/src/main/scala/org/apache/griffin/measure/cache/ZKCacheLock.scala b/measure/src/main/scala/org/apache/griffin/measure/cache/lock/ZKCacheLock.scala similarity index 96% rename from measure/src/main/scala/org/apache/griffin/measure/cache/ZKCacheLock.scala rename to measure/src/main/scala/org/apache/griffin/measure/cache/lock/ZKCacheLock.scala index 0c24ac679..1ac9ad2ce 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/cache/ZKCacheLock.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/cache/lock/ZKCacheLock.scala @@ -16,7 +16,7 @@ KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License. */ -package org.apache.griffin.measure.cache +package org.apache.griffin.measure.cache.lock import java.util.concurrent.TimeUnit diff --git a/measure/src/main/scala/org/apache/griffin/measure/cache/result/CacheResult.scala b/measure/src/main/scala/org/apache/griffin/measure/cache/result/CacheResult.scala new file mode 100644 index 000000000..92787be4e --- /dev/null +++ b/measure/src/main/scala/org/apache/griffin/measure/cache/result/CacheResult.scala @@ -0,0 +1,29 @@ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ +package org.apache.griffin.measure.cache.result + +import org.apache.griffin.measure.result.Result + +case class CacheResult(timeGroup: Long, updateTime: Long, result: Result) { + + def olderThan(ut: Long): Boolean = { + updateTime < ut + } + +} diff --git a/measure/src/main/scala/org/apache/griffin/measure/cache/result/CacheResultProcesser.scala b/measure/src/main/scala/org/apache/griffin/measure/cache/result/CacheResultProcesser.scala new file mode 100644 index 000000000..9e389258d --- /dev/null +++ b/measure/src/main/scala/org/apache/griffin/measure/cache/result/CacheResultProcesser.scala @@ -0,0 +1,73 @@ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ +package org.apache.griffin.measure.cache.result + +import org.apache.griffin.measure.log.Loggable +import org.apache.griffin.measure.result._ + +import scala.collection.mutable.{Map => MutableMap} + +case class CacheResultProcesser() extends Loggable { + + private val cacheGroup: MutableMap[Long, CacheResult] = MutableMap() + + def genUpdateCacheResult(timeGroup: Long, updateTime: Long, result: Result): Option[CacheResult] = { + cacheGroup.get(timeGroup) match { + case Some(cr) => { + if (cr.olderThan(updateTime)) { + val existResult = cr.result + val newResult = existResult.update(result.asInstanceOf[existResult.T]) + if (existResult.differsFrom(newResult)) { + Some(CacheResult(timeGroup, updateTime, newResult)) + } else None + } else None + } + case _ => { + Some(CacheResult(timeGroup, updateTime, result)) + } + } + } + + def update(cr: CacheResult): Unit = { + if (!cr.result.eventual) { + val t = cr.timeGroup + cacheGroup.get(t) match { + case Some(c) => { + if (c.olderThan(t)) cacheGroup += (t -> cr) + } + case _ => cacheGroup += (t -> cr) + } + } + } + + def getCacheResult(timeGroup: Long): Option[CacheResult] = { + cacheGroup.get(timeGroup) + } + + def refresh(overtime: Long): Unit = { + val curCacheGroup = cacheGroup.toMap + val deadCache = curCacheGroup.filter { pr => + val (_, cr) = pr + cr.timeGroup <= overtime || cr.result.eventual() + } + info(s"=== dead cache group count: ${deadCache.size} ===") + deadCache.keySet.foreach(cacheGroup -= _) + } + +} diff --git a/measure/src/main/scala/org/apache/griffin/measure/connector/CacheDataConnector.scala b/measure/src/main/scala/org/apache/griffin/measure/connector/CacheDataConnector.scala index 32ecb7bc1..2a327c605 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/connector/CacheDataConnector.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/connector/CacheDataConnector.scala @@ -18,7 +18,7 @@ under the License. */ package org.apache.griffin.measure.connector -import org.apache.griffin.measure.cache.{InfoCacheInstance, ZKInfoCache} +import org.apache.griffin.measure.cache.info.{InfoCacheInstance, ZKInfoCache} import org.apache.griffin.measure.utils.TimeUtil import org.apache.spark.sql.DataFrame diff --git a/measure/src/main/scala/org/apache/griffin/measure/rule/RuleAnalyzer.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/RuleAnalyzer.scala index bade43e26..56d07c770 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rule/RuleAnalyzer.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rule/RuleAnalyzer.scala @@ -46,6 +46,8 @@ case class RuleAnalyzer(rule: StatementExpr) extends Serializable { val targetRuleExprs: RuleExprs = RuleExprs(targetGroupbyExprs, targetCacheExprs, targetFinalCacheExprs, targetPersistExprs) + val whenClauseExprOpt: Option[WhenClauseExpr] = rule.whenClauseOpt + } @@ -58,8 +60,6 @@ case class RuleAnalyzer(rule: StatementExpr) extends Serializable { // Cached for the calculation usage, and can be saved for the re-calculation in streaming mode // persistExprs: the expr values should be persisted, only the direct selection exprs are persistable // Persisted for record usage, to record the missing data, need be readable as raw data -// whenClauseExprOpt: when clause of rule, to determine if the row of data source is filtered -// Can be prep-calculated to filter some data in data connector case class RuleExprs(groupbyExprs: Seq[Expr], cacheExprs: Iterable[Expr], finalCacheExprs: Iterable[Expr], @@ -71,5 +71,4 @@ case class RuleExprs(groupbyExprs: Seq[Expr], // cacheExprs: $target.name, $target.age, $target.age + (3 * 4) // finalCacheExprs: $target.name, $target.age + (3 * 4), $target.age // persistExprs: $target.name, $target.age - // whenClauseExprOpt: None } \ No newline at end of file diff --git a/measure/src/test/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgoTest.scala b/measure/src/test/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgoTest.scala index 2e6cb72c6..befe28bcd 100644 --- a/measure/src/test/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgoTest.scala +++ b/measure/src/test/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgoTest.scala @@ -22,7 +22,8 @@ import java.util.Date import java.util.concurrent.TimeUnit import org.apache.griffin.measure.algo.batch.BatchAccuracyAlgo -import org.apache.griffin.measure.cache.InfoCacheInstance +import org.apache.griffin.measure.cache.info.InfoCacheInstance +import org.apache.griffin.measure.cache.result._ import org.apache.griffin.measure.config.params._ import org.apache.griffin.measure.config.params.env._ import org.apache.griffin.measure.config.params.user._ @@ -31,6 +32,7 @@ import org.apache.griffin.measure.config.validator._ import org.apache.griffin.measure.connector.{BatchDataConnector, DataConnector, DataConnectorFactory} import org.apache.griffin.measure.log.Loggable import org.apache.griffin.measure.persist.{Persist, PersistFactory} +import org.apache.griffin.measure.result._ import org.apache.griffin.measure.rule.expr._ import org.apache.griffin.measure.rule.{ExprValueUtil, RuleAnalyzer, RuleFactory} import org.apache.griffin.measure.utils.TimeUtil @@ -164,6 +166,8 @@ class StreamingAccuracyAlgoTest extends FunSuite with Matchers with BeforeAndAft InfoCacheInstance.initInstance(envParam.infoCacheParams, metricName) InfoCacheInstance.init + val cacheResultProcesser = CacheResultProcesser() + // init data stream sourceDataConnector.init() targetDataConnector.init() @@ -175,6 +179,7 @@ class StreamingAccuracyAlgoTest extends FunSuite with Matchers with BeforeAndAft case class Process() extends Runnable { val lock = InfoCacheInstance.genLock("process") def run(): Unit = { + val updateTime = new Date().getTime val locked = lock.lock(5, TimeUnit.SECONDS) if (locked) { try { @@ -196,8 +201,72 @@ class StreamingAccuracyAlgoTest extends FunSuite with Matchers with BeforeAndAft val (accuResult, missingRdd, matchedRdd) = algo.accuracy(sourceData, targetData, ruleAnalyzer) println(accuResult) - val et = new Date().getTime - println(s"process time: ${et - st} ms") + // result of every group + val matchedGroups = algo.reorgByTimeGroup(matchedRdd) + val matchedGroupCount = matchedGroups.count + println(s"===== matchedGroupCount: ${matchedGroupCount} =====") + + // get missing results + val missingGroups = algo.reorgByTimeGroup(missingRdd) + val missingGroupCount = missingGroups.count + println(s"===== missingGroupCount: ${missingGroupCount} =====") + + val groups = matchedGroups.cogroup(missingGroups) + val groupCount = groups.count + println(s"===== groupCount: ${groupCount} =====") + + val updateResults = groups.flatMap { group => + val (t, (matchData, missData)) = group + + val matchSize = matchData.size + val missSize = missData.size + val res = AccuracyResult(missSize, matchSize + missSize) + + val updatedCacheResulOpt = cacheResultProcesser.genUpdateCacheResult(t, updateTime, res) + + // updated result + if (updatedCacheResulOpt.nonEmpty) { + val missStrings = missData.map { row => + val (key, (value, info)) = row + s"${value} [${info.getOrElse(MismatchInfo.key, "unknown")}]" + } + // persist missing data + missStrings.foreach(println) + // record missing records +// try { +// persist.accuracyMissingRecords(missStrings) +// } catch { +// case e: Throwable => println("missing record error: " + e.getMessage) +// } + } + + updatedCacheResulOpt + }.collect() + + // persist update results + updateResults.foreach { updateResult => + // cache + cacheResultProcesser.update(updateResult) + + // persist + // fixme: + println(updateResult) + } + + // dump missing rdd +// ruleAnalyzer.whenClauseExprOpt +// val savingRdd: RDD[(Product, (Map[String, Any], Map[String, Any]))] = missingRdd.filter { row => +// val (key, (value, info)) = row +// info.get(TimeGroupInfo.key) match { +// case Some(t: Long) => { +// cacheProcesser.getCache(t) match { +// case Some(cache) => true +// case _ => false +// } +// } +// case _ => false +// } +// } // persist time // persist.log(et, s"calculation using time: ${et - st} ms") @@ -209,6 +278,8 @@ class StreamingAccuracyAlgoTest extends FunSuite with Matchers with BeforeAndAft // val pet = new Date().getTime // persist.log(pet, s"persist using time: ${pet - et} ms") + val et = new Date().getTime + println(s"process time: ${et - st} ms") } finally { lock.unlock() } diff --git a/measure/src/test/scala/org/apache/griffin/measure/cache/InfoCacheInstanceTest.scala b/measure/src/test/scala/org/apache/griffin/measure/cache/InfoCacheInstanceTest.scala index 1458f0960..f754e3fb7 100644 --- a/measure/src/test/scala/org/apache/griffin/measure/cache/InfoCacheInstanceTest.scala +++ b/measure/src/test/scala/org/apache/griffin/measure/cache/InfoCacheInstanceTest.scala @@ -24,6 +24,7 @@ import java.util.concurrent.{Executors, ThreadPoolExecutor, TimeUnit} import org.apache.curator.framework.recipes.locks.InterProcessMutex import org.apache.curator.framework.{CuratorFramework, CuratorFrameworkFactory} import org.apache.curator.retry.ExponentialBackoffRetry +import org.apache.griffin.measure.cache.info.InfoCacheInstance import org.apache.griffin.measure.config.params.env.InfoCacheParam import org.junit.runner.RunWith import org.scalatest.junit.JUnitRunner diff --git a/measure/src/test/scala/org/apache/griffin/measure/cache/ZKCacheLockTest.scala b/measure/src/test/scala/org/apache/griffin/measure/cache/ZKCacheLockTest.scala index 8e3dffc35..8fb400dc7 100644 --- a/measure/src/test/scala/org/apache/griffin/measure/cache/ZKCacheLockTest.scala +++ b/measure/src/test/scala/org/apache/griffin/measure/cache/ZKCacheLockTest.scala @@ -24,6 +24,7 @@ import java.util.concurrent.{Executors, ThreadPoolExecutor, TimeUnit} import org.apache.curator.framework.recipes.locks.InterProcessMutex import org.apache.curator.framework.{CuratorFramework, CuratorFrameworkFactory} import org.apache.curator.retry.ExponentialBackoffRetry +import org.apache.griffin.measure.cache.info.ZKInfoCache import org.junit.runner.RunWith import org.scalatest.junit.JUnitRunner import org.scalatest.{BeforeAndAfter, FunSuite, Matchers} diff --git a/measure/src/test/scala/org/apache/griffin/measure/cache/ZKInfoCacheTest.scala b/measure/src/test/scala/org/apache/griffin/measure/cache/ZKInfoCacheTest.scala index 5e9795713..8bd79952d 100644 --- a/measure/src/test/scala/org/apache/griffin/measure/cache/ZKInfoCacheTest.scala +++ b/measure/src/test/scala/org/apache/griffin/measure/cache/ZKInfoCacheTest.scala @@ -24,6 +24,7 @@ import java.util.concurrent.{Executors, ThreadPoolExecutor, TimeUnit} import org.apache.curator.framework.recipes.locks.InterProcessMutex import org.apache.curator.framework.{CuratorFramework, CuratorFrameworkFactory} import org.apache.curator.retry.ExponentialBackoffRetry +import org.apache.griffin.measure.cache.info.ZKInfoCache import org.junit.runner.RunWith import org.scalatest.junit.JUnitRunner import org.scalatest.{BeforeAndAfter, FunSuite, Matchers} diff --git a/measure/src/test/scala/org/apache/griffin/measure/connector/ConnectorTest.scala b/measure/src/test/scala/org/apache/griffin/measure/connector/ConnectorTest.scala index 0f3890438..9b36b6a27 100644 --- a/measure/src/test/scala/org/apache/griffin/measure/connector/ConnectorTest.scala +++ b/measure/src/test/scala/org/apache/griffin/measure/connector/ConnectorTest.scala @@ -23,7 +23,7 @@ import java.util.concurrent.TimeUnit import kafka.serializer.StringDecoder import org.apache.griffin.measure.algo.streaming.StreamingProcess -import org.apache.griffin.measure.cache.InfoCacheInstance +import org.apache.griffin.measure.cache.info.InfoCacheInstance import org.apache.griffin.measure.config.params.env._ import org.apache.griffin.measure.config.params.user.{DataConnectorParam, EvaluateRuleParam} import org.apache.griffin.measure.config.reader.ParamRawStringReader diff --git a/service/src/main/resources/sparkJob.properties b/service/src/main/resources/sparkJob.properties index 819f00d39..51969ac31 100644 --- a/service/src/main/resources/sparkJob.properties +++ b/service/src/main/resources/sparkJob.properties @@ -1,4 +1,4 @@ -ssparkJob.file=hdfs:///griffin/griffin-measure.jar +sparkJob.file=hdfs:///griffin/griffin-measure.jar sparkJob.className=org.apache.griffin.measure.Application sparkJob.args_1=hdfs:///griffin/json/env.json sparkJob.args_3=hdfs,raw From c0c8ab1da3b9729da2b151063602159552cd8014 Mon Sep 17 00:00:00 2001 From: Lionel Liu Date: Mon, 17 Jul 2017 16:23:35 +0800 Subject: [PATCH 022/111] v1 --- .../config/params/user/DataCacheParam.scala | 31 ++++++++++++++++ .../params/user/DataConnectorParam.scala | 13 ++----- .../connector/DataConnectorFactory.scala | 13 +++---- .../connector/DfCacheDataConnector.scala | 5 ++- .../connector/KafkaDataConnector.scala | 9 ++--- .../griffin/measure/rule/RuleAnalyzer.scala | 2 -- .../src/test/resources/config-streaming.json | 36 ++++++++++--------- .../measure/connector/ConnectorTest.scala | 19 ++++------ 8 files changed, 71 insertions(+), 57 deletions(-) create mode 100644 measure/src/main/scala/org/apache/griffin/measure/config/params/user/DataCacheParam.scala diff --git a/measure/src/main/scala/org/apache/griffin/measure/config/params/user/DataCacheParam.scala b/measure/src/main/scala/org/apache/griffin/measure/config/params/user/DataCacheParam.scala new file mode 100644 index 000000000..9c607558a --- /dev/null +++ b/measure/src/main/scala/org/apache/griffin/measure/config/params/user/DataCacheParam.scala @@ -0,0 +1,31 @@ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ +package org.apache.griffin.measure.config.params.user + +import com.fasterxml.jackson.annotation.{JsonInclude, JsonProperty} +import com.fasterxml.jackson.annotation.JsonInclude.Include +import org.apache.griffin.measure.config.params.Param + +@JsonInclude(Include.NON_NULL) +case class DataCacheParam( @JsonProperty("type") cacheType: String, + @JsonProperty("config") config: Map[String, Any], + @JsonProperty("time.range") timeRange: List[String] + ) extends Param { + +} diff --git a/measure/src/main/scala/org/apache/griffin/measure/config/params/user/DataConnectorParam.scala b/measure/src/main/scala/org/apache/griffin/measure/config/params/user/DataConnectorParam.scala index 233c15228..66b33d5a6 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/config/params/user/DataConnectorParam.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/config/params/user/DataConnectorParam.scala @@ -25,17 +25,8 @@ import org.apache.griffin.measure.config.params.Param @JsonInclude(Include.NON_NULL) case class DataConnectorParam( @JsonProperty("type") conType: String, @JsonProperty("version") version: String, - @JsonProperty("config") config: Map[String, Any] + @JsonProperty("config") config: Map[String, Any], + @JsonProperty("cache") cache: DataCacheParam ) extends Param { -} - -object DataConnectorParam { - def apply(map: Map[String, Any]): DataConnectorParam = { - DataConnectorParam( - map.getOrElse("type", "").toString, - map.getOrElse("version", "").toString, - map.getOrElse("config", Map[String, Any]()).asInstanceOf[Map[String, Any]] - ) - } } \ No newline at end of file diff --git a/measure/src/main/scala/org/apache/griffin/measure/connector/DataConnectorFactory.scala b/measure/src/main/scala/org/apache/griffin/measure/connector/DataConnectorFactory.scala index f543a4d95..add1da3a1 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/connector/DataConnectorFactory.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/connector/DataConnectorFactory.scala @@ -91,15 +91,16 @@ object DataConnectorFactory { } def getCacheDataConnector(sqlContext: SQLContext, - dataConnectorParam: DataConnectorParam + dataCacheParam: DataCacheParam ): Try[CacheDataConnector] = { - val conType = dataConnectorParam.conType - val version = dataConnectorParam.version - val config = dataConnectorParam.config + if (dataCacheParam == null) { + throw new Exception("invalid data cache param!") + } + val cacheType = dataCacheParam.cacheType Try { - conType match { + cacheType match { case DfRegex() => { - DfCacheDataConnector(sqlContext, config) + DfCacheDataConnector(sqlContext, dataCacheParam) } case _ => throw new Exception("cache connector creation error!") } diff --git a/measure/src/main/scala/org/apache/griffin/measure/connector/DfCacheDataConnector.scala b/measure/src/main/scala/org/apache/griffin/measure/connector/DfCacheDataConnector.scala index 0f302227f..f0148bd32 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/connector/DfCacheDataConnector.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/connector/DfCacheDataConnector.scala @@ -18,6 +18,7 @@ under the License. */ package org.apache.griffin.measure.connector +import org.apache.griffin.measure.config.params.user.DataCacheParam import org.apache.griffin.measure.result.TimeStampInfo import org.apache.griffin.measure.utils.TimeUtil import org.apache.spark.rdd.RDD @@ -26,9 +27,11 @@ import org.apache.spark.storage.StorageLevel import scala.util.{Success, Try} -case class DfCacheDataConnector(sqlContext: SQLContext, config: Map[String, Any] +case class DfCacheDataConnector(sqlContext: SQLContext, dataCacheParam: DataCacheParam ) extends CacheDataConnector { + val config = dataCacheParam.config + val CacheLevel = "cache.level" val cacheLevel: String = config.getOrElse(CacheLevel, "MEMORY_ONLY").toString diff --git a/measure/src/main/scala/org/apache/griffin/measure/connector/KafkaDataConnector.scala b/measure/src/main/scala/org/apache/griffin/measure/connector/KafkaDataConnector.scala index c3687906c..159da9807 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/connector/KafkaDataConnector.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/connector/KafkaDataConnector.scala @@ -41,13 +41,8 @@ case class KafkaDataConnector(sqlContext: SQLContext, @transient ssc: StreamingC case Failure(ex) => throw ex } - val Cache = "cache" - - val cacheDataConnectorParam = dataConnectorParam.config.get(Cache) match { - case Some(map: Map[String, Any]) => DataConnectorParam(map) - case _ => throw new Exception("invalid cache parameter!") - } - val cacheDataConnector = DataConnectorFactory.getCacheDataConnector(sqlContext, cacheDataConnectorParam) match { + val dataCacheParam = dataConnectorParam.cache + val cacheDataConnector = DataConnectorFactory.getCacheDataConnector(sqlContext, dataCacheParam) match { case Success(cntr) => cntr case Failure(ex) => throw ex } diff --git a/measure/src/main/scala/org/apache/griffin/measure/rule/RuleAnalyzer.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/RuleAnalyzer.scala index 56d07c770..5ec143fb6 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rule/RuleAnalyzer.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rule/RuleAnalyzer.scala @@ -46,8 +46,6 @@ case class RuleAnalyzer(rule: StatementExpr) extends Serializable { val targetRuleExprs: RuleExprs = RuleExprs(targetGroupbyExprs, targetCacheExprs, targetFinalCacheExprs, targetPersistExprs) - val whenClauseExprOpt: Option[WhenClauseExpr] = rule.whenClauseOpt - } diff --git a/measure/src/test/resources/config-streaming.json b/measure/src/test/resources/config-streaming.json index a7a04e768..459d51c04 100644 --- a/measure/src/test/resources/config-streaming.json +++ b/measure/src/test/resources/config-streaming.json @@ -14,14 +14,15 @@ }, "topics": "sss", "key.type": "java.lang.String", - "value.type": "java.lang.String", - "cache": { - "type": "df", - "config": { - "cache.level": "MEMORY_AND_DISK", - "info.path": "src" - } - } + "value.type": "java.lang.String" + }, + "cache": { + "type": "df", + "config": { + "cache.level": "MEMORY_AND_DISK", + "info.path": "src" + }, + "time.range": ["-2m", "0"] } }, @@ -37,19 +38,20 @@ }, "topics": "ttt", "key.type": "java.lang.String", - "value.type": "java.lang.String", - "cache": { - "type": "df", - "config": { - "cache.level": "MEMORY_AND_DISK", - "info.path": "tgt" - } - } + "value.type": "java.lang.String" + }, + "cache": { + "type": "df", + "config": { + "cache.level": "MEMORY_AND_DISK", + "info.path": "tgt" + }, + "time.range": [] } }, "evaluateRule": { "sampleRatio": 0.2, - "rules": "$source.json().name = $target.json().name AND $source.json().age = $target.json().age WHEN $source._tmst_ < $target._tmst_ + 2m" + "rules": "$source.json().name = $target.json().name AND $source.json().age = $target.json().age" } } \ No newline at end of file diff --git a/measure/src/test/scala/org/apache/griffin/measure/connector/ConnectorTest.scala b/measure/src/test/scala/org/apache/griffin/measure/connector/ConnectorTest.scala index 9b36b6a27..5e3285da6 100644 --- a/measure/src/test/scala/org/apache/griffin/measure/connector/ConnectorTest.scala +++ b/measure/src/test/scala/org/apache/griffin/measure/connector/ConnectorTest.scala @@ -25,7 +25,7 @@ import kafka.serializer.StringDecoder import org.apache.griffin.measure.algo.streaming.StreamingProcess import org.apache.griffin.measure.cache.info.InfoCacheInstance import org.apache.griffin.measure.config.params.env._ -import org.apache.griffin.measure.config.params.user.{DataConnectorParam, EvaluateRuleParam} +import org.apache.griffin.measure.config.params.user.{DataCacheParam, DataConnectorParam, EvaluateRuleParam} import org.apache.griffin.measure.config.reader.ParamRawStringReader import org.apache.griffin.measure.result.{DataInfo, TimeStampInfo} import org.apache.griffin.measure.rule.expr.{Expr, StatementExpr} @@ -106,17 +106,13 @@ class ConnectorTest extends FunSuite with Matchers with BeforeAndAfter { ("info.path" -> "src") ) - val cacheParam = Map[String, Any]( - ("type" -> "df"), - ("config" -> cacheConfig) - ) + val cacheParam = DataCacheParam("df", cacheConfig, Nil) val config = Map[String, Any]( ("kafka.config" -> kafkaConfig), ("topics" -> "sss"), ("key.type" -> "java.lang.String"), - ("value.type" -> "java.lang.String"), - ("cache" -> cacheParam) + ("value.type" -> "java.lang.String") ) val infoCacheConfig = Map[String, Any]( @@ -136,7 +132,7 @@ class ConnectorTest extends FunSuite with Matchers with BeforeAndAfter { InfoCacheInstance.init - val connectorParam = DataConnectorParam("kafka", "0.8", config) + val connectorParam = DataConnectorParam("kafka", "0.8", config, cacheParam) val conf = new SparkConf().setMaster("local[*]").setAppName("ConnectorTest") val sc = new SparkContext(conf) @@ -157,11 +153,8 @@ class ConnectorTest extends FunSuite with Matchers with BeforeAndAfter { case _ => fail } - val cacheDataConnectorParam = connectorParam.config.get("cache") match { - case Some(map: Map[String, Any]) => DataConnectorParam(map) - case _ => throw new Exception("invalid cache parameter!") - } - val cacheDataConnector = DataConnectorFactory.getCacheDataConnector(sqlContext, cacheDataConnectorParam) match { + val dataCacheParam = connectorParam.cache + val cacheDataConnector = DataConnectorFactory.getCacheDataConnector(sqlContext, dataCacheParam) match { case Success(cntr) => cntr case Failure(ex) => throw ex } From 5a9e4bec1b4caa34c443af7146362a170d8d4912 Mon Sep 17 00:00:00 2001 From: Lionel Liu Date: Tue, 18 Jul 2017 18:15:16 +0800 Subject: [PATCH 023/111] t10 --- .../streaming/StreamingAccuracyAlgo.scala | 2 +- ...amingProcess.scala => TimingProcess.scala} | 2 +- .../measure/cache/info/InfoCache.scala | 2 + .../cache/info/InfoCacheInstance.scala | 6 + .../measure/cache/info/TimeInfoCache.scala | 91 ++++++++ .../measure/cache/info/ZKInfoCache.scala | 6 + .../measure/cache/lock/ZKCacheLock.scala | 6 +- .../cache/result/CacheResultProcesser.scala | 12 +- .../config/params/env/CleanerParam.scala | 3 +- .../params/user/DataConnectorParam.scala | 7 +- .../connector/CacheDataConnector.scala | 39 ++-- .../measure/connector/DataConnector.scala | 7 + .../connector/DfCacheDataConnector.scala | 195 ++++++++++++++++-- .../connector/KafkaDataConnector.scala | 10 + .../griffin/measure/utils/TimeUtil.scala | 7 +- .../src/test/resources/config-streaming.json | 16 +- measure/src/test/resources/env-streaming.json | 2 +- .../streaming/StreamingAccuracyAlgoTest.scala | 59 ++++-- .../measure/connector/ConnectorTest.scala | 6 +- 19 files changed, 389 insertions(+), 89 deletions(-) rename measure/src/main/scala/org/apache/griffin/measure/algo/streaming/{StreamingProcess.scala => TimingProcess.scala} (95%) create mode 100644 measure/src/main/scala/org/apache/griffin/measure/cache/info/TimeInfoCache.scala diff --git a/measure/src/main/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgo.scala b/measure/src/main/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgo.scala index 0099235a4..5d81ba55f 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgo.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgo.scala @@ -157,7 +157,7 @@ case class StreamingAccuracyAlgo(allParam: AllParam) extends AccuracyAlgo { case Some(interval) => interval case _ => throw new Exception("invalid batch interval") } - val process = StreamingProcess(processInterval, Process()) + val process = TimingProcess(processInterval, Process()) process.startup() diff --git a/measure/src/main/scala/org/apache/griffin/measure/algo/streaming/StreamingProcess.scala b/measure/src/main/scala/org/apache/griffin/measure/algo/streaming/TimingProcess.scala similarity index 95% rename from measure/src/main/scala/org/apache/griffin/measure/algo/streaming/StreamingProcess.scala rename to measure/src/main/scala/org/apache/griffin/measure/algo/streaming/TimingProcess.scala index 28fd07067..e5bd7deff 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/algo/streaming/StreamingProcess.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/algo/streaming/TimingProcess.scala @@ -21,7 +21,7 @@ package org.apache.griffin.measure.algo.streaming import java.util.concurrent.{Executors, ThreadPoolExecutor, TimeUnit} import java.util.{Timer, TimerTask} -case class StreamingProcess(interval: Long, runnable: Runnable) { +case class TimingProcess(interval: Long, runnable: Runnable) { val pool: ThreadPoolExecutor = Executors.newFixedThreadPool(5).asInstanceOf[ThreadPoolExecutor] diff --git a/measure/src/main/scala/org/apache/griffin/measure/cache/info/InfoCache.scala b/measure/src/main/scala/org/apache/griffin/measure/cache/info/InfoCache.scala index f7250a4ec..8e4b25d9e 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/cache/info/InfoCache.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/cache/info/InfoCache.scala @@ -32,6 +32,8 @@ trait InfoCache extends Loggable with Serializable { def deleteInfo(keys: Iterable[String]): Unit def clearInfo(): Unit + def listKeys(path: String): List[String] + def genLock(s: String): CacheLock } diff --git a/measure/src/main/scala/org/apache/griffin/measure/cache/info/InfoCacheInstance.scala b/measure/src/main/scala/org/apache/griffin/measure/cache/info/InfoCacheInstance.scala index 4bb50a549..bcde26610 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/cache/info/InfoCacheInstance.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/cache/info/InfoCacheInstance.scala @@ -43,5 +43,11 @@ object InfoCacheInstance extends InfoCache { def deleteInfo(keys: Iterable[String]): Unit = infoCaches.foreach(_.deleteInfo(keys)) def clearInfo(): Unit = infoCaches.foreach(_.clearInfo) + def listKeys(path: String): List[String] = { + infoCaches.foldLeft(Nil: List[String]) { (res, infoCache) => + if (res.size > 0) res else infoCache.listKeys(path) + } + } + def genLock(s: String): CacheLock = MultiCacheLock(infoCaches.map(_.genLock(s))) } \ No newline at end of file diff --git a/measure/src/main/scala/org/apache/griffin/measure/cache/info/TimeInfoCache.scala b/measure/src/main/scala/org/apache/griffin/measure/cache/info/TimeInfoCache.scala new file mode 100644 index 000000000..cdd179853 --- /dev/null +++ b/measure/src/main/scala/org/apache/griffin/measure/cache/info/TimeInfoCache.scala @@ -0,0 +1,91 @@ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ +package org.apache.griffin.measure.cache.info + +import org.apache.griffin.measure.log.Loggable + +object TimeInfoCache extends Loggable with Serializable { + + private val CacheTime = "cache.time" + private val LastProcTime = "last.proc.time" + private val ReadyTime = "ready.time" + + def cacheTime(path: String): String = s"${path}/${CacheTime}" + def lastProcTime(path: String): String = s"${path}/${LastProcTime}" + def readyTime(path: String): String = s"${path}/${ReadyTime}" + + val infoPath = "info" + + val finalCacheInfoPath = "info.final" + val finalReadyTime = s"${finalCacheInfoPath}/${ReadyTime}" + val finalLastProcTime = s"${finalCacheInfoPath}/${LastProcTime}" + + def startTimeInfoCache(): Unit = { + genFinalReadyTime + } + + def getTimeRange(): (Long, Long) = { + readTimeRange + } + + def endTimeInfoCache: Unit = { + genFinalLastProcTime + } + + private def genFinalReadyTime(): Unit = { + val subPath = InfoCacheInstance.listKeys(infoPath) + val keys = subPath.map { p => s"${infoPath}/${p}/${ReadyTime}" } + val result = InfoCacheInstance.readInfo(keys) + val time = keys.map { k => + getLong(result, k) + }.min + val map = Map[String, String]((finalReadyTime -> time.toString)) + InfoCacheInstance.cacheInfo(map) + } + + private def genFinalLastProcTime(): Unit = { + val subPath = InfoCacheInstance.listKeys(infoPath) + val keys = subPath.map { p => s"${infoPath}/${p}/${LastProcTime}" } + val result = InfoCacheInstance.readInfo(keys) + val time = keys.map { k => + getLong(result, k) + }.min + val map = Map[String, String]((finalLastProcTime -> time.toString)) + InfoCacheInstance.cacheInfo(map) + } + + private def readTimeRange(): (Long, Long) = { + val map = InfoCacheInstance.readInfo(List(finalLastProcTime, finalReadyTime)) + val lastProcTime = getLong(map, finalLastProcTime) + val curReadyTime = getLong(map, finalReadyTime) + (lastProcTime + 1, curReadyTime) + } + + private def getLong(map: Map[String, String], key: String): Long = { + try { + map.get(key) match { + case Some(v) => v.toLong + case _ => -1 + } + } catch { + case _ => -1 + } + } + +} diff --git a/measure/src/main/scala/org/apache/griffin/measure/cache/info/ZKInfoCache.scala b/measure/src/main/scala/org/apache/griffin/measure/cache/info/ZKInfoCache.scala index 5417b3a9c..479243806 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/cache/info/ZKInfoCache.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/cache/info/ZKInfoCache.scala @@ -25,6 +25,8 @@ import org.apache.curator.retry.ExponentialBackoffRetry import org.apache.griffin.measure.cache.lock.ZKCacheLock import org.apache.zookeeper.CreateMode +import scala.collection.JavaConverters._ + case class ZKInfoCache(config: Map[String, Any], metricName: String) extends InfoCache { val Hosts = "hosts" @@ -112,6 +114,10 @@ case class ZKInfoCache(config: Map[String, Any], metricName: String) extends Inf delete("/") } + def listKeys(p: String): List[String] = { + client.getChildren().forPath(path(p)).asScala.toList + } + def genLock(s: String): ZKCacheLock = { val lpt = if (s.isEmpty) path(lockPath) else path(lockPath) + separator + s ZKCacheLock(new InterProcessMutex(client, lpt)) diff --git a/measure/src/main/scala/org/apache/griffin/measure/cache/lock/ZKCacheLock.scala b/measure/src/main/scala/org/apache/griffin/measure/cache/lock/ZKCacheLock.scala index 1ac9ad2ce..77ee83bef 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/cache/lock/ZKCacheLock.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/cache/lock/ZKCacheLock.scala @@ -26,7 +26,11 @@ case class ZKCacheLock(@transient mutex: InterProcessMutex) extends CacheLock { def lock(outtime: Long, unit: TimeUnit): Boolean = { try { - mutex.acquire(outtime, unit) + if (outtime >= 0) { + mutex.acquire(outtime, unit) + } else { + mutex.acquire(-1, null) + } } catch { case e: Throwable => { error(s"lock error: ${e.getMessage}") diff --git a/measure/src/main/scala/org/apache/griffin/measure/cache/result/CacheResultProcesser.scala b/measure/src/main/scala/org/apache/griffin/measure/cache/result/CacheResultProcesser.scala index 9e389258d..a8251bae5 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/cache/result/CacheResultProcesser.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/cache/result/CacheResultProcesser.scala @@ -45,14 +45,12 @@ case class CacheResultProcesser() extends Loggable { } def update(cr: CacheResult): Unit = { - if (!cr.result.eventual) { - val t = cr.timeGroup - cacheGroup.get(t) match { - case Some(c) => { - if (c.olderThan(t)) cacheGroup += (t -> cr) - } - case _ => cacheGroup += (t -> cr) + val t = cr.timeGroup + cacheGroup.get(t) match { + case Some(c) => { + if (c.olderThan(cr.updateTime)) cacheGroup += (t -> cr) } + case _ => cacheGroup += (t -> cr) } } diff --git a/measure/src/main/scala/org/apache/griffin/measure/config/params/env/CleanerParam.scala b/measure/src/main/scala/org/apache/griffin/measure/config/params/env/CleanerParam.scala index 6c1ff9494..00302a776 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/config/params/env/CleanerParam.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/config/params/env/CleanerParam.scala @@ -23,6 +23,7 @@ import com.fasterxml.jackson.annotation.JsonInclude.Include import org.apache.griffin.measure.config.params.Param @JsonInclude(Include.NON_NULL) -case class CleanerParam() extends Param { +case class CleanerParam( @JsonProperty("clean.interval") cleanInterval: String + ) extends Param { } diff --git a/measure/src/main/scala/org/apache/griffin/measure/config/params/user/DataConnectorParam.scala b/measure/src/main/scala/org/apache/griffin/measure/config/params/user/DataConnectorParam.scala index 66b33d5a6..dbc2e0baf 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/config/params/user/DataConnectorParam.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/config/params/user/DataConnectorParam.scala @@ -26,7 +26,12 @@ import org.apache.griffin.measure.config.params.Param case class DataConnectorParam( @JsonProperty("type") conType: String, @JsonProperty("version") version: String, @JsonProperty("config") config: Map[String, Any], - @JsonProperty("cache") cache: DataCacheParam + @JsonProperty("cache") cache: DataCacheParam, + @JsonProperty("match.once") matchOnce: Boolean ) extends Param { + def getMatchOnce(): Boolean = { + if (matchOnce == null) false else matchOnce + } + } \ No newline at end of file diff --git a/measure/src/main/scala/org/apache/griffin/measure/connector/CacheDataConnector.scala b/measure/src/main/scala/org/apache/griffin/measure/connector/CacheDataConnector.scala index 2a327c605..df30a970e 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/connector/CacheDataConnector.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/connector/CacheDataConnector.scala @@ -18,7 +18,7 @@ under the License. */ package org.apache.griffin.measure.connector -import org.apache.griffin.measure.cache.info.{InfoCacheInstance, ZKInfoCache} +import org.apache.griffin.measure.cache.info.{InfoCacheInstance, TimeInfoCache, ZKInfoCache} import org.apache.griffin.measure.utils.TimeUtil import org.apache.spark.sql.DataFrame @@ -34,43 +34,28 @@ trait CacheDataConnector extends DataConnector { val readyTimeInterval: Long val readyTimeDelay: Long - def CacheTime = s"${cacheInfoPath}/cache.time" - def LastProcTime = s"${cacheInfoPath}/last.proc.time" - def ReadyTime = s"${cacheInfoPath}/ready.time" + def selfCacheInfoPath = s"${TimeInfoCache.infoPath}/${cacheInfoPath}" + + def selfCacheTime = TimeInfoCache.cacheTime(selfCacheInfoPath) + def selfLastProcTime = TimeInfoCache.lastProcTime(selfCacheInfoPath) + def selfReadyTime = TimeInfoCache.readyTime(selfCacheInfoPath) protected def submitCacheTime(ms: Long): Unit = { - val map = Map[String, String]((CacheTime -> ms.toString)) + val map = Map[String, String]((selfCacheTime -> ms.toString)) InfoCacheInstance.cacheInfo(map) } protected def submitReadyTime(ms: Long): Unit = { - if (ms % readyTimeInterval == 0) { - val map = Map[String, String]((ReadyTime -> ms.toString)) + val curReadyTime = ms - readyTimeDelay + if (curReadyTime % readyTimeInterval == 0) { + val map = Map[String, String]((selfReadyTime -> curReadyTime.toString)) InfoCacheInstance.cacheInfo(map) } } - def submitLastProcTime(ms: Long): Unit = { - val map = Map[String, String]((LastProcTime -> ms.toString)) + protected def submitLastProcTime(ms: Long): Unit = { + val map = Map[String, String]((selfLastProcTime -> ms.toString)) InfoCacheInstance.cacheInfo(map) } - protected def readTimeRange(): (Long, Long) = { - val map = InfoCacheInstance.readInfo(List(LastProcTime, ReadyTime)) - val lastProcTime = getLong(map, LastProcTime) - val curReadyTime = getLong(map, ReadyTime) - (lastProcTime + 1, curReadyTime) - } - - private def getLong(map: Map[String, String], key: String): Long = { - try { - map.get(key) match { - case Some(v) => v.toLong - case _ => -1 - } - } catch { - case _ => -1 - } - } - } diff --git a/measure/src/main/scala/org/apache/griffin/measure/connector/DataConnector.scala b/measure/src/main/scala/org/apache/griffin/measure/connector/DataConnector.scala index 3f76fe39b..13f4ab996 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/connector/DataConnector.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/connector/DataConnector.scala @@ -19,6 +19,8 @@ under the License. package org.apache.griffin.measure.connector import org.apache.griffin.measure.log.Loggable +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.DataFrame trait DataConnector extends Loggable with Serializable { @@ -27,4 +29,9 @@ trait DataConnector extends Loggable with Serializable { def init(): Unit = {} + def cleanOldData(): Unit = {} + + def updateOldData(oldDf: DataFrame): Unit = {} + def updateOldData(oldRdd: RDD[Map[String, Any]]): Unit = {} + } diff --git a/measure/src/main/scala/org/apache/griffin/measure/connector/DfCacheDataConnector.scala b/measure/src/main/scala/org/apache/griffin/measure/connector/DfCacheDataConnector.scala index f0148bd32..839be659a 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/connector/DfCacheDataConnector.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/connector/DfCacheDataConnector.scala @@ -18,6 +18,11 @@ under the License. */ package org.apache.griffin.measure.connector +import java.util.concurrent.TimeUnit +import java.util.concurrent.atomic.AtomicLong + +import org.apache.griffin.measure.cache.info.{InfoCacheInstance, TimeInfoCache} +import org.apache.griffin.measure.cache.lock.CacheLock import org.apache.griffin.measure.config.params.user.DataCacheParam import org.apache.griffin.measure.result.TimeStampInfo import org.apache.griffin.measure.utils.TimeUtil @@ -31,17 +36,36 @@ case class DfCacheDataConnector(sqlContext: SQLContext, dataCacheParam: DataCach ) extends CacheDataConnector { val config = dataCacheParam.config + val InfoPath = "info.path" + val cacheInfoPath: String = config.getOrElse(InfoPath, PathCounter.genPath).toString + + val newCacheLock = InfoCacheInstance.genLock(s"${cacheInfoPath}.new") + val oldCacheLock = InfoCacheInstance.genLock(s"${cacheInfoPath}.old") + + val timeRangeParam: List[String] = if (dataCacheParam.timeRange != null) dataCacheParam.timeRange else Nil + val deltaTimeRange: (Long, Long) = (timeRangeParam ::: List("0", "0")) match { + case s :: e :: _ => { + val ns = TimeUtil.milliseconds(s) match { + case Some(n) if (n < 0) => n + case _ => 0 + } + val ne = TimeUtil.milliseconds(e) match { + case Some(n) if (n < 0) => n + case _ => 0 + } + (ns, ne) + } + case _ => (0, 0) + } + println(deltaTimeRange) val CacheLevel = "cache.level" val cacheLevel: String = config.getOrElse(CacheLevel, "MEMORY_ONLY").toString val timeStampColumn = TimeStampInfo.key - var initialed: Boolean = false - var dataFrame: DataFrame = _ - - val InfoPath = "info.path" - val cacheInfoPath: String = config.getOrElse(InfoPath, "path").toString + var newDataFrame: DataFrame = null + var oldDataFrame: DataFrame = null val ReadyTimeInterval = "ready.time.interval" val ReadyTimeDelay = "ready.time.delay" @@ -53,31 +77,158 @@ case class DfCacheDataConnector(sqlContext: SQLContext, dataCacheParam: DataCach } def saveData(df: DataFrame, ms: Long): Unit = { - if (!initialed) { - dataFrame = df - dataFrame.persist(StorageLevel.fromString(cacheLevel)) - initialed = true - } else { - if (!df.rdd.isEmpty) { - dataFrame.unpersist() - dataFrame = dataFrame.unionAll(df) - dataFrame.persist(StorageLevel.fromString(cacheLevel)) + val newCacheLocked = newCacheLock.lock(-1, TimeUnit.SECONDS) + if (newCacheLocked) { + try { + if (newDataFrame == null) { + newDataFrame = df + newDataFrame.persist(StorageLevel.fromString(cacheLevel)) + } else { + if (!df.rdd.isEmpty) { + newDataFrame.unpersist() + newDataFrame = newDataFrame.unionAll(df) + newDataFrame.persist(StorageLevel.fromString(cacheLevel)) + } + } + // submit ms + submitCacheTime(ms) + submitReadyTime(ms) + } finally { + newCacheLock.unlock() } } - - // submit ms - submitCacheTime(ms) - submitReadyTime(ms) +// if (newDataFrame == null) { +// newDataFrame = df +// newDataFrame.persist(StorageLevel.fromString(cacheLevel)) +// } else { +// if (!df.rdd.isEmpty) { +// newDataFrame.unpersist() +// newDataFrame = newDataFrame.unionAll(df) +// newDataFrame.persist(StorageLevel.fromString(cacheLevel)) +// } +// } +// // submit ms +// submitCacheTime(ms) +// submitReadyTime(ms) } def readData(): Try[DataFrame] = Try { - if (initialed) { - val timeRange = readTimeRange - submitLastProcTime(timeRange._2) - dataFrame.filter(s"${timeStampColumn} BETWEEN ${timeRange._1} AND ${timeRange._2}") +// if (initialed) { +// val timeRange = TimeInfoCache.getTimeRange +// println(s"timeRange: ${timeRange}") +// submitLastProcTime(timeRange._2) +// +// val reviseTimeRange = (timeRange._1 + deltaTimeRange._1, timeRange._2 + deltaTimeRange._2) +// println(s"reviseTimeRange: ${reviseTimeRange}") +// dataFrame.filter(s"${timeStampColumn} BETWEEN ${reviseTimeRange._1} AND ${reviseTimeRange._2}") +// } else { +// throw new Exception("data not cached") +// } + + val timeRange = TimeInfoCache.getTimeRange + println(s"timeRange: ${timeRange}") + submitLastProcTime(timeRange._2) + val reviseTimeRange = (timeRange._1 + deltaTimeRange._1, timeRange._2 + deltaTimeRange._2) + println(s"reviseTimeRange: ${reviseTimeRange}") + + // move new data frame to temp data frame + val newCacheLocked = newCacheLock.lock(-1, TimeUnit.SECONDS) + val newTempDataFrame = if (newCacheLocked) { + try { + val tmp = newDataFrame.filter(s"${timeStampColumn} BETWEEN ${reviseTimeRange._1} AND ${reviseTimeRange._2}") + newDataFrame.unpersist() + newDataFrame = newDataFrame.filter(s"${timeStampColumn} > ${reviseTimeRange._2}") + tmp + } finally { + newCacheLock.unlock() + } + } else null + + // add temp data frame to old data frame + val oldCacheLocked = oldCacheLock.lock(-1, TimeUnit.SECONDS) + val oldTempDataFrame = if (oldCacheLocked) { + try { + if (oldDataFrame != null) { + oldDataFrame.filter(s"${timeStampColumn} BETWEEN ${reviseTimeRange._1} AND ${reviseTimeRange._2}") + } else null + } finally { + oldCacheLock.unlock() + } } else { + throw new Exception("old cache lock unavailable") + } + + if (oldTempDataFrame == null && newTempDataFrame == null) { throw new Exception("data not cached") + } else { + val finalDataFrame = if (newTempDataFrame == null) { + oldTempDataFrame + } else if (oldTempDataFrame == null) { + newTempDataFrame + } else { + oldTempDataFrame.unionAll(newTempDataFrame) + } + finalDataFrame + } + } + + override def cleanOldData(): Unit = { + val oldCacheLocked = oldCacheLock.lock(-1, TimeUnit.SECONDS) + if (oldCacheLocked) { + try { + val timeRange = TimeInfoCache.getTimeRange + val reviseTimeRange = (timeRange._1 + deltaTimeRange._1, timeRange._2 + deltaTimeRange._2) + println(s"clean reviseTimeRange: ${reviseTimeRange}") + + oldDataFrame.unpersist() + oldDataFrame = oldDataFrame.filter(s"${timeStampColumn} >= ${reviseTimeRange._1}") + oldDataFrame.persist(StorageLevel.fromString(cacheLevel)) + } finally { + oldCacheLock.unlock() + } + } + +// if (initialed) { +// val timeRange = TimeInfoCache.getTimeRange +// val reviseTimeRange = (timeRange._1 + deltaTimeRange._1, timeRange._2 + deltaTimeRange._2) +// println(s"clean reviseTimeRange: ${reviseTimeRange}") +// +// dataFrame.show(10) +// +// dataFrame.unpersist() +// dataFrame = dataFrame.filter(s"${timeStampColumn} >= ${reviseTimeRange._1}") +// dataFrame.persist(StorageLevel.fromString(cacheLevel)) +// +// dataFrame.show(10) +// } + } + + override def updateOldData(oldDf: DataFrame): Unit = { + val oldCacheLocked = oldCacheLock.lock(-1, TimeUnit.SECONDS) + if (oldCacheLocked) { + try { + if (oldDataFrame == null) { + oldDataFrame = oldDf + oldDataFrame.persist(StorageLevel.fromString(cacheLevel)) + } else { + if (!oldDf.rdd.isEmpty) { + oldDataFrame.unpersist() + oldDataFrame = oldDf + oldDataFrame.persist(StorageLevel.fromString(cacheLevel)) + } + } + } finally { + oldCacheLock.unlock() + } } } } + +object PathCounter { + private val counter: AtomicLong = new AtomicLong(0L) + def genPath(): String = s"path${increment}" + private def increment(): Long = { + counter.incrementAndGet() + } +} \ No newline at end of file diff --git a/measure/src/main/scala/org/apache/griffin/measure/connector/KafkaDataConnector.scala b/measure/src/main/scala/org/apache/griffin/measure/connector/KafkaDataConnector.scala index 159da9807..b3c70cc6f 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/connector/KafkaDataConnector.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/connector/KafkaDataConnector.scala @@ -172,6 +172,16 @@ case class KafkaDataConnector(sqlContext: SQLContext, @transient ssc: StreamingC } } + override def cleanOldData(): Unit = { + cacheDataConnector.cleanOldData + } + + override def updateOldData(oldRdd: RDD[Map[String, Any]]): Unit = { + if (dataConnectorParam.getMatchOnce) { + cacheDataConnector.updateOldData(genDataFrame(oldRdd)) + } + } + // private def dbPrefix(): Boolean = { // if (useTempTable) false else dumpDatabase.nonEmpty && !dumpDatabase.equals("default") // } diff --git a/measure/src/main/scala/org/apache/griffin/measure/utils/TimeUtil.scala b/measure/src/main/scala/org/apache/griffin/measure/utils/TimeUtil.scala index fa8a0ea16..0079d104c 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/utils/TimeUtil.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/utils/TimeUtil.scala @@ -22,7 +22,8 @@ import scala.util.{Failure, Success, Try} object TimeUtil { - final val TimeRegex = """(\d+)(d|h|m|s|ms)""".r + final val TimeRegex = """([+\-]?\d+)(d|h|m|s|ms)""".r + final val PureTimeRegex = """([+\-]?\d+)""".r def milliseconds(timeString: String): Option[Long] = { val value: Option[Long] = { @@ -39,6 +40,10 @@ object TimeUtil { case _ => throw new Exception(s"${timeString} is invalid time format") } } + case PureTimeRegex(time) => { + val t = time.toLong + t + } case _ => throw new Exception(s"${timeString} is invalid time format") } } match { diff --git a/measure/src/test/resources/config-streaming.json b/measure/src/test/resources/config-streaming.json index 459d51c04..d827f0f7b 100644 --- a/measure/src/test/resources/config-streaming.json +++ b/measure/src/test/resources/config-streaming.json @@ -20,10 +20,13 @@ "type": "df", "config": { "cache.level": "MEMORY_AND_DISK", - "info.path": "src" + "info.path": "src", + "ready.time.interval": "10s", + "ready.time.delay": "0" }, "time.range": ["-2m", "0"] - } + }, + "match.once": true }, "target": { @@ -44,10 +47,13 @@ "type": "df", "config": { "cache.level": "MEMORY_AND_DISK", - "info.path": "tgt" + "info.path": "tgt", + "ready.time.interval": "10s", + "ready.time.delay": "0" }, - "time.range": [] - } + "time.range": ["-2m", "0"] + }, + "match.once": false }, "evaluateRule": { diff --git a/measure/src/test/resources/env-streaming.json b/measure/src/test/resources/env-streaming.json index 4d72d9b0a..be6fa7ea7 100644 --- a/measure/src/test/resources/env-streaming.json +++ b/measure/src/test/resources/env-streaming.json @@ -40,6 +40,6 @@ ], "cleaner": { - + "clean.interval": "2m" } } \ No newline at end of file diff --git a/measure/src/test/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgoTest.scala b/measure/src/test/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgoTest.scala index befe28bcd..2c692b260 100644 --- a/measure/src/test/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgoTest.scala +++ b/measure/src/test/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgoTest.scala @@ -22,7 +22,7 @@ import java.util.Date import java.util.concurrent.TimeUnit import org.apache.griffin.measure.algo.batch.BatchAccuracyAlgo -import org.apache.griffin.measure.cache.info.InfoCacheInstance +import org.apache.griffin.measure.cache.info.{InfoCacheInstance, TimeInfoCache} import org.apache.griffin.measure.cache.result._ import org.apache.griffin.measure.config.params._ import org.apache.griffin.measure.config.params.env._ @@ -104,6 +104,7 @@ class StreamingAccuracyAlgoTest extends FunSuite with Matchers with BeforeAndAft val userParam = allParam.userParam val metricName = userParam.name val sparkParam = envParam.sparkParam + val cleanerParam = envParam.cleanerParam val batchInterval = TimeUtil.milliseconds(sparkParam.batchInterval) match { case Some(interval) => Milliseconds(interval) @@ -128,6 +129,9 @@ class StreamingAccuracyAlgoTest extends FunSuite with Matchers with BeforeAndAft // persist start id // persist.start(applicationId) + InfoCacheInstance.initInstance(envParam.infoCacheParams, metricName) + InfoCacheInstance.init + // generate rule from rule param, generate rule analyzer val ruleFactory = RuleFactory(userParam.evaluateRuleParam) val rule: StatementExpr = ruleFactory.generateRule() @@ -163,9 +167,6 @@ class StreamingAccuracyAlgoTest extends FunSuite with Matchers with BeforeAndAft case Failure(ex) => throw ex } - InfoCacheInstance.initInstance(envParam.infoCacheParams, metricName) - InfoCacheInstance.init - val cacheResultProcesser = CacheResultProcesser() // init data stream @@ -184,6 +185,9 @@ class StreamingAccuracyAlgoTest extends FunSuite with Matchers with BeforeAndAft if (locked) { try { val st = new Date().getTime + + TimeInfoCache.startTimeInfoCache + // get data val sourceData = sourceDataConnector.data match { case Success(dt) => dt @@ -254,19 +258,12 @@ class StreamingAccuracyAlgoTest extends FunSuite with Matchers with BeforeAndAft } // dump missing rdd -// ruleAnalyzer.whenClauseExprOpt -// val savingRdd: RDD[(Product, (Map[String, Any], Map[String, Any]))] = missingRdd.filter { row => -// val (key, (value, info)) = row -// info.get(TimeGroupInfo.key) match { -// case Some(t: Long) => { -// cacheProcesser.getCache(t) match { -// case Some(cache) => true -// case _ => false -// } -// } -// case _ => false -// } -// } + val dumpRdd: RDD[Map[String, Any]] = missingRdd.map { row => + val (key, (value, info)) = row + value ++ info + } + sourceDataConnector.updateOldData(dumpRdd) + targetDataConnector.updateOldData(dumpRdd) // persist time // persist.log(et, s"calculation using time: ${et - st} ms") @@ -278,6 +275,9 @@ class StreamingAccuracyAlgoTest extends FunSuite with Matchers with BeforeAndAft // val pet = new Date().getTime // persist.log(pet, s"persist using time: ${pet - et} ms") + + TimeInfoCache.endTimeInfoCache + val et = new Date().getTime println(s"process time: ${et - st} ms") } finally { @@ -291,9 +291,31 @@ class StreamingAccuracyAlgoTest extends FunSuite with Matchers with BeforeAndAft case Some(interval) => interval case _ => throw new Exception("invalid batch interval") } - val process = StreamingProcess(processInterval, Process()) + val process = TimingProcess(processInterval, Process()) + + // clean thread +// case class Clean() extends Runnable { +// val lock = InfoCacheInstance.genLock("clean") +// def run(): Unit = { +// val locked = lock.lock(5, TimeUnit.SECONDS) +// if (locked) { +// try { +// sourceDataConnector.cleanData +// targetDataConnector.cleanData +// } finally { +// lock.unlock() +// } +// } +// } +// } +// val cleanInterval = TimeUtil.milliseconds(cleanerParam.cleanInterval) match { +// case Some(interval) => interval +// case _ => throw new Exception("invalid batch interval") +// } +// val clean = TimingProcess(cleanInterval, Clean()) process.startup() +// clean.startup() ssc.start() ssc.awaitTermination() @@ -307,6 +329,7 @@ class StreamingAccuracyAlgoTest extends FunSuite with Matchers with BeforeAndAft // persist.finish() process.shutdown() +// clean.shutdown() } private def readParamFile[T <: Param](file: String, fsType: String)(implicit m : Manifest[T]): Try[T] = { diff --git a/measure/src/test/scala/org/apache/griffin/measure/connector/ConnectorTest.scala b/measure/src/test/scala/org/apache/griffin/measure/connector/ConnectorTest.scala index 5e3285da6..64b5c0743 100644 --- a/measure/src/test/scala/org/apache/griffin/measure/connector/ConnectorTest.scala +++ b/measure/src/test/scala/org/apache/griffin/measure/connector/ConnectorTest.scala @@ -22,7 +22,7 @@ import java.util.Date import java.util.concurrent.TimeUnit import kafka.serializer.StringDecoder -import org.apache.griffin.measure.algo.streaming.StreamingProcess +import org.apache.griffin.measure.algo.streaming.TimingProcess import org.apache.griffin.measure.cache.info.InfoCacheInstance import org.apache.griffin.measure.config.params.env._ import org.apache.griffin.measure.config.params.user.{DataCacheParam, DataConnectorParam, EvaluateRuleParam} @@ -132,7 +132,7 @@ class ConnectorTest extends FunSuite with Matchers with BeforeAndAfter { InfoCacheInstance.init - val connectorParam = DataConnectorParam("kafka", "0.8", config, cacheParam) + val connectorParam = DataConnectorParam("kafka", "0.8", config, cacheParam, false) val conf = new SparkConf().setMaster("local[*]").setAppName("ConnectorTest") val sc = new SparkContext(conf) @@ -292,7 +292,7 @@ class ConnectorTest extends FunSuite with Matchers with BeforeAndAfter { case Some(interval) => interval case _ => throw new Exception("invalid batch interval") } - val process = StreamingProcess(processInterval, Process()) + val process = TimingProcess(processInterval, Process()) process.startup() From 396ed7065107afca64d44217ea15f091f9ffcd86 Mon Sep 17 00:00:00 2001 From: Lionel Liu Date: Wed, 19 Jul 2017 11:26:12 +0800 Subject: [PATCH 024/111] t11 --- .../algo/batch/BatchAccuracyAlgo.scala | 3 +- .../measure/algo/batch/BatchProfileAlgo.scala | 5 +- .../connector/CacheDataConnector.scala | 12 ++++ .../connector/DfCacheDataConnector.scala | 11 +-- .../connector/HiveCacheDataConnector.scala | 32 +++++++++ .../griffin/measure/persist/HdfsPersist.scala | 47 +++++++++++-- .../griffin/measure/persist/HttpPersist.scala | 7 +- .../measure/persist/LoggerPersist.scala | 49 ++++++++++--- .../measure/persist/MultiPersists.scala | 7 +- .../griffin/measure/persist/Persist.scala | 12 +++- measure/src/test/resources/env-streaming.json | 13 +--- .../streaming/StreamingAccuracyAlgoTest.scala | 68 +++++++++---------- 12 files changed, 187 insertions(+), 79 deletions(-) create mode 100644 measure/src/main/scala/org/apache/griffin/measure/connector/HiveCacheDataConnector.scala diff --git a/measure/src/main/scala/org/apache/griffin/measure/algo/batch/BatchAccuracyAlgo.scala b/measure/src/main/scala/org/apache/griffin/measure/algo/batch/BatchAccuracyAlgo.scala index f63b9bdb6..28e89ef6f 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/algo/batch/BatchAccuracyAlgo.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/algo/batch/BatchAccuracyAlgo.scala @@ -127,7 +127,8 @@ case class BatchAccuracyAlgo(allParam: AllParam) extends AccuracyAlgo { // persist result persist.result(endTime, accuResult) val missingRecords = missingRdd.map(record2String(_, ruleAnalyzer.sourceRuleExprs.persistExprs, ruleAnalyzer.targetRuleExprs.persistExprs)) - persist.missRecords(missingRecords) +// persist.missRecords(missingRecords) + persist.records(missingRecords, PersistType.MISS) // persist end time val persistEndTime = new Date().getTime diff --git a/measure/src/main/scala/org/apache/griffin/measure/algo/batch/BatchProfileAlgo.scala b/measure/src/main/scala/org/apache/griffin/measure/algo/batch/BatchProfileAlgo.scala index 6b9cc564b..1672e61e7 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/algo/batch/BatchProfileAlgo.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/algo/batch/BatchProfileAlgo.scala @@ -24,7 +24,7 @@ import org.apache.griffin.measure.algo.ProfileAlgo import org.apache.griffin.measure.algo.core.ProfileCore import org.apache.griffin.measure.config.params._ import org.apache.griffin.measure.connector._ -import org.apache.griffin.measure.persist.{Persist, PersistFactory} +import org.apache.griffin.measure.persist.{Persist, PersistFactory, PersistType} import org.apache.griffin.measure.result._ import org.apache.griffin.measure.rule.expr._ import org.apache.griffin.measure.rule.{ExprValueUtil, RuleAnalyzer, RuleFactory} @@ -109,7 +109,8 @@ case class BatchProfileAlgo(allParam: AllParam) extends ProfileAlgo { // persist result persist.result(endTime, profileResult) val matchedRecords = matchedRdd.map(record2String(_, ruleAnalyzer.sourceRuleExprs.persistExprs)) - persist.matchRecords(matchedRecords) +// persist.matchRecords(matchedRecords) + persist.records(matchedRecords, PersistType.MATCH) // persist end time val persistEndTime = new Date().getTime diff --git a/measure/src/main/scala/org/apache/griffin/measure/connector/CacheDataConnector.scala b/measure/src/main/scala/org/apache/griffin/measure/connector/CacheDataConnector.scala index df30a970e..1c42c7054 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/connector/CacheDataConnector.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/connector/CacheDataConnector.scala @@ -18,6 +18,8 @@ under the License. */ package org.apache.griffin.measure.connector +import java.util.concurrent.atomic.AtomicLong + import org.apache.griffin.measure.cache.info.{InfoCacheInstance, TimeInfoCache, ZKInfoCache} import org.apache.griffin.measure.utils.TimeUtil import org.apache.spark.sql.DataFrame @@ -26,6 +28,8 @@ import scala.util.Try trait CacheDataConnector extends DataConnector { + protected val defCacheInfoPath = PathCounter.genPath + def saveData(df: DataFrame, ms: Long): Unit def readData(): Try[DataFrame] @@ -59,3 +63,11 @@ trait CacheDataConnector extends DataConnector { } } + +object PathCounter { + private val counter: AtomicLong = new AtomicLong(0L) + def genPath(): String = s"path_${increment}" + private def increment(): Long = { + counter.incrementAndGet() + } +} \ No newline at end of file diff --git a/measure/src/main/scala/org/apache/griffin/measure/connector/DfCacheDataConnector.scala b/measure/src/main/scala/org/apache/griffin/measure/connector/DfCacheDataConnector.scala index 839be659a..678f56d3c 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/connector/DfCacheDataConnector.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/connector/DfCacheDataConnector.scala @@ -19,7 +19,6 @@ under the License. package org.apache.griffin.measure.connector import java.util.concurrent.TimeUnit -import java.util.concurrent.atomic.AtomicLong import org.apache.griffin.measure.cache.info.{InfoCacheInstance, TimeInfoCache} import org.apache.griffin.measure.cache.lock.CacheLock @@ -37,7 +36,7 @@ case class DfCacheDataConnector(sqlContext: SQLContext, dataCacheParam: DataCach val config = dataCacheParam.config val InfoPath = "info.path" - val cacheInfoPath: String = config.getOrElse(InfoPath, PathCounter.genPath).toString + val cacheInfoPath: String = config.getOrElse(InfoPath, defCacheInfoPath).toString val newCacheLock = InfoCacheInstance.genLock(s"${cacheInfoPath}.new") val oldCacheLock = InfoCacheInstance.genLock(s"${cacheInfoPath}.old") @@ -223,12 +222,4 @@ case class DfCacheDataConnector(sqlContext: SQLContext, dataCacheParam: DataCach } } -} - -object PathCounter { - private val counter: AtomicLong = new AtomicLong(0L) - def genPath(): String = s"path${increment}" - private def increment(): Long = { - counter.incrementAndGet() - } } \ No newline at end of file diff --git a/measure/src/main/scala/org/apache/griffin/measure/connector/HiveCacheDataConnector.scala b/measure/src/main/scala/org/apache/griffin/measure/connector/HiveCacheDataConnector.scala new file mode 100644 index 000000000..6db66778c --- /dev/null +++ b/measure/src/main/scala/org/apache/griffin/measure/connector/HiveCacheDataConnector.scala @@ -0,0 +1,32 @@ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ +package org.apache.griffin.measure.connector + +import org.apache.griffin.measure.config.params.user.DataCacheParam +import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.hive.HiveContext + +case class HiveCacheDataConnector(sqlContext: SQLContext, dataCacheParam: DataCacheParam + ) extends CacheDataConnector { + + if (!sqlContext.isInstanceOf[HiveContext]) { + throw new Exception("hive context not prepared!") + } + +} diff --git a/measure/src/main/scala/org/apache/griffin/measure/persist/HdfsPersist.scala b/measure/src/main/scala/org/apache/griffin/measure/persist/HdfsPersist.scala index 68a5fb899..ff6bb67f1 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/persist/HdfsPersist.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/persist/HdfsPersist.scala @@ -146,14 +146,53 @@ case class HdfsPersist(config: Map[String, Any], metricName: String, timeStamp: } } - def missRecords(records: RDD[String]): Unit = { - rddRecords(records, MissRecFile) + private def iterableRecords(records: Iterable[String], path: String): Unit = { + try { + val recordCount = records.size + val count = if (maxPersistLines < 0) recordCount else scala.math.min(maxPersistLines, recordCount) + if (count > 0) { + val groupCount = ((count - 1) / maxLinesPerFile + 1).toInt + if (groupCount <= 1) { + val recs = records.take(count.toInt) + persistRecords(path, recs) + } else { + val groupedRecords = records.grouped(groupCount).zipWithIndex + groupedRecords.take(groupCount).foreach { group => + val (recs, gid) = group + val hdfsPath = if (gid == 0) path else withSuffix(path, gid.toString) + persistRecords(hdfsPath, recs) + } + } + } + } catch { + case e: Throwable => error(e.getMessage) + } } - def matchRecords(records: RDD[String]): Unit = { - rddRecords(records, MatchRecFile) + def records(recs: RDD[String], tp: String): Unit = { + tp match { + case PersistType.MISS => rddRecords(recs, MissRecFile) + case PersistType.MATCH => rddRecords(recs, MatchRecFile) + case _ => {} + } } + def records(recs: Iterable[String], tp: String): Unit = { + tp match { + case PersistType.MISS => iterableRecords(recs, MissRecFile) + case PersistType.MATCH => iterableRecords(recs, MatchRecFile) + case _ => {} + } + } + +// def missRecords(records: RDD[String]): Unit = { +// rddRecords(records, MissRecFile) +// } +// +// def matchRecords(records: RDD[String]): Unit = { +// rddRecords(records, MatchRecFile) +// } + private def persistRecords(hdfsPath: String, records: Iterable[String]): Unit = { val recStr = records.mkString("\n") HdfsUtil.appendContent(hdfsPath, recStr) diff --git a/measure/src/main/scala/org/apache/griffin/measure/persist/HttpPersist.scala b/measure/src/main/scala/org/apache/griffin/measure/persist/HttpPersist.scala index 0b2ed3adc..ef90912d3 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/persist/HttpPersist.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/persist/HttpPersist.scala @@ -70,8 +70,11 @@ case class HttpPersist(config: Map[String, Any], metricName: String, timeStamp: } - def missRecords(records: RDD[String]): Unit = {} - def matchRecords(records: RDD[String]): Unit = {} + def records(recs: RDD[String], tp: String): Unit = {} + def records(recs: Iterable[String], tp: String): Unit = {} + +// def missRecords(records: RDD[String]): Unit = {} +// def matchRecords(records: RDD[String]): Unit = {} def log(rt: Long, msg: String): Unit = {} diff --git a/measure/src/main/scala/org/apache/griffin/measure/persist/LoggerPersist.scala b/measure/src/main/scala/org/apache/griffin/measure/persist/LoggerPersist.scala index 71c457fef..a066ddf85 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/persist/LoggerPersist.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/persist/LoggerPersist.scala @@ -34,10 +34,10 @@ case class LoggerPersist(config: Map[String, Any], metricName: String, timeStamp def available(): Boolean = true def start(msg: String): Unit = { - info(s"${metricName} start") + println(s"[${timeStamp}] ${metricName} start") } def finish(): Unit = { - info(s"${metricName} finish") + println(s"[${timeStamp}] ${metricName} finish") } def result(rt: Long, result: Result): Unit = { @@ -53,7 +53,7 @@ case class LoggerPersist(config: Map[String, Any], metricName: String, timeStamp s"result: ${result}" } } - info(s"${metricName} result: \n${resStr}") + println(s"[${timeStamp}] ${metricName} result: \n${resStr}") } catch { case e: Throwable => error(e.getMessage) } @@ -73,17 +73,46 @@ case class LoggerPersist(config: Map[String, Any], metricName: String, timeStamp } } - def missRecords(records: RDD[String]): Unit = { - info(s"${metricName} miss records: ") - rddRecords(records) + private def iterableRecords(records: Iterable[String]): Unit = { + try { + val recordCount = records.size + val count = if (maxLogLines < 0) recordCount else scala.math.min(maxLogLines, recordCount) + if (count > 0) { + val recordsArray = records.take(count) + recordsArray.foreach(println) + } + } catch { + case e: Throwable => error(e.getMessage) + } } - def matchRecords(records: RDD[String]): Unit = { - info(s"${metricName} match records: ") - rddRecords(records) + + def records(recs: RDD[String], tp: String): Unit = { + tp match { + case PersistType.MISS => rddRecords(recs) + case PersistType.MATCH => rddRecords(recs) + case _ => {} + } } + def records(recs: Iterable[String], tp: String): Unit = { + tp match { + case PersistType.MISS => iterableRecords(recs) + case PersistType.MATCH => iterableRecords(recs) + case _ => {} + } + } + +// def missRecords(records: RDD[String]): Unit = { +// warn(s"[${timeStamp}] ${metricName} miss records: ") +// rddRecords(records) +// } +// def matchRecords(records: RDD[String]): Unit = { +// warn(s"[${timeStamp}] ${metricName} match records: ") +// rddRecords(records) +// } + def log(rt: Long, msg: String): Unit = { - info(s"${rt}: ${msg}") + println(s"[${timeStamp}] ${rt}: ${msg}") } } diff --git a/measure/src/main/scala/org/apache/griffin/measure/persist/MultiPersists.scala b/measure/src/main/scala/org/apache/griffin/measure/persist/MultiPersists.scala index de30ad062..25c8b0bb7 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/persist/MultiPersists.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/persist/MultiPersists.scala @@ -41,8 +41,11 @@ case class MultiPersists(persists: Iterable[Persist]) extends Persist { def result(rt: Long, result: Result): Unit = { persists.foreach(_.result(rt, result)) } - def missRecords(records: RDD[String]): Unit = { persists.foreach(_.missRecords(records)) } - def matchRecords(records: RDD[String]): Unit = { persists.foreach(_.matchRecords(records)) } + def records(recs: RDD[String], tp: String): Unit = { persists.foreach(_.records(recs, tp)) } + def records(recs: Iterable[String], tp: String): Unit = { persists.foreach(_.records(recs, tp)) } + +// def missRecords(records: RDD[String]): Unit = { persists.foreach(_.missRecords(records)) } +// def matchRecords(records: RDD[String]): Unit = { persists.foreach(_.matchRecords(records)) } def log(rt: Long, msg: String): Unit = { persists.foreach(_.log(rt, msg)) } diff --git a/measure/src/main/scala/org/apache/griffin/measure/persist/Persist.scala b/measure/src/main/scala/org/apache/griffin/measure/persist/Persist.scala index 01f205d97..bc16599c6 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/persist/Persist.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/persist/Persist.scala @@ -37,8 +37,16 @@ trait Persist extends Loggable with Serializable { def result(rt: Long, result: Result): Unit - def missRecords(records: RDD[String]): Unit - def matchRecords(records: RDD[String]): Unit + def records(recs: RDD[String], tp: String): Unit + def records(recs: Iterable[String], tp: String): Unit + +// def missRecords(records: RDD[String]): Unit +// def matchRecords(records: RDD[String]): Unit def log(rt: Long, msg: String): Unit } + +object PersistType { + final val MISS = "miss" + final val MATCH = "match" +} \ No newline at end of file diff --git a/measure/src/test/resources/env-streaming.json b/measure/src/test/resources/env-streaming.json index be6fa7ea7..761a4a527 100644 --- a/measure/src/test/resources/env-streaming.json +++ b/measure/src/test/resources/env-streaming.json @@ -9,18 +9,9 @@ "persist": [ { - "type": "hdfs", + "type": "log", "config": { - "path": "hdfs:///griffin/streaming/persist", - "max.persist.lines": 10000, - "max.lines.per.file": 10000 - } - }, - { - "type": "http", - "config": { - "method": "post", - "api": "http://HOSTNAME:9200/griffin/accuracy" + "max.log.lines": 100 } } ], diff --git a/measure/src/test/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgoTest.scala b/measure/src/test/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgoTest.scala index 2c692b260..866534c24 100644 --- a/measure/src/test/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgoTest.scala +++ b/measure/src/test/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgoTest.scala @@ -31,7 +31,7 @@ import org.apache.griffin.measure.config.reader._ import org.apache.griffin.measure.config.validator._ import org.apache.griffin.measure.connector.{BatchDataConnector, DataConnector, DataConnectorFactory} import org.apache.griffin.measure.log.Loggable -import org.apache.griffin.measure.persist.{Persist, PersistFactory} +import org.apache.griffin.measure.persist.{Persist, PersistFactory, PersistType} import org.apache.griffin.measure.result._ import org.apache.griffin.measure.rule.expr._ import org.apache.griffin.measure.rule.{ExprValueUtil, RuleAnalyzer, RuleFactory} @@ -120,14 +120,16 @@ class StreamingAccuracyAlgoTest extends FunSuite with Matchers with BeforeAndAft // start time val startTime = new Date().getTime() + val persistFactory = PersistFactory(envParam.persistParams, metricName) + // get persists to persist measure result -// val persist: Persist = PersistFactory(envParam.persistParams, metricName).getPersists(startTime) + val appPersist: Persist = persistFactory.getPersists(startTime) // get spark application id val applicationId = sc.applicationId // persist start id -// persist.start(applicationId) + appPersist.start(applicationId) InfoCacheInstance.initInstance(envParam.infoCacheParams, metricName) InfoCacheInstance.init @@ -205,6 +207,9 @@ class StreamingAccuracyAlgoTest extends FunSuite with Matchers with BeforeAndAft val (accuResult, missingRdd, matchedRdd) = algo.accuracy(sourceData, targetData, ruleAnalyzer) println(accuResult) + val ct = new Date().getTime + appPersist.log(ct, s"calculation using time: ${ct - st} ms") + // result of every group val matchedGroups = algo.reorgByTimeGroup(matchedRdd) val matchedGroupCount = matchedGroups.count @@ -226,35 +231,33 @@ class StreamingAccuracyAlgoTest extends FunSuite with Matchers with BeforeAndAft val missSize = missData.size val res = AccuracyResult(missSize, matchSize + missSize) - val updatedCacheResulOpt = cacheResultProcesser.genUpdateCacheResult(t, updateTime, res) + val updatedCacheResultOpt = cacheResultProcesser.genUpdateCacheResult(t, updateTime, res) + + updatedCacheResultOpt match { + case Some(updatedCacheResult) => { + val persist: Persist = persistFactory.getPersists(t) + + // persist result + persist.result(updateTime, accuResult) - // updated result - if (updatedCacheResulOpt.nonEmpty) { - val missStrings = missData.map { row => - val (key, (value, info)) = row - s"${value} [${info.getOrElse(MismatchInfo.key, "unknown")}]" + // persist missing data + val missStrings = missData.map { row => + val (key, (value, info)) = row + s"${value} [${info.getOrElse(MismatchInfo.key, "unknown")}]" + } + persist.records(missStrings, PersistType.MISS) + } + case _ => { + // do nothing } - // persist missing data - missStrings.foreach(println) - // record missing records -// try { -// persist.accuracyMissingRecords(missStrings) -// } catch { -// case e: Throwable => println("missing record error: " + e.getMessage) -// } } - updatedCacheResulOpt + updatedCacheResultOpt }.collect() - // persist update results + // update results cache together updateResults.foreach { updateResult => - // cache cacheResultProcesser.update(updateResult) - - // persist - // fixme: - println(updateResult) } // dump missing rdd @@ -263,23 +266,18 @@ class StreamingAccuracyAlgoTest extends FunSuite with Matchers with BeforeAndAft value ++ info } sourceDataConnector.updateOldData(dumpRdd) - targetDataConnector.updateOldData(dumpRdd) + targetDataConnector.updateOldData(dumpRdd) // not correct - // persist time - // persist.log(et, s"calculation using time: ${et - st} ms") + // fixme: 2. implement the hive cahe data connector - // persist result - // persist.result(et, accuResult) -// val missingRecords = missingRdd.map(algo.record2String(_, ruleAnalyzer.sourceRuleExprs.persistExprs, ruleAnalyzer.targetRuleExprs.persistExprs)) - // persist.missRecords(missingRecords) + // fixme: 3. refactor data connector module - // val pet = new Date().getTime - // persist.log(pet, s"persist using time: ${pet - et} ms") TimeInfoCache.endTimeInfoCache val et = new Date().getTime - println(s"process time: ${et - st} ms") + appPersist.log(et, s"persist using time: ${et - ct} ms") + } finally { lock.unlock() } @@ -326,7 +324,7 @@ class StreamingAccuracyAlgoTest extends FunSuite with Matchers with BeforeAndAft InfoCacheInstance.close -// persist.finish() + appPersist.finish() process.shutdown() // clean.shutdown() From d6237fb0958dd9626517b8431924f31f832e269f Mon Sep 17 00:00:00 2001 From: Lionel Liu Date: Thu, 20 Jul 2017 09:21:04 +0800 Subject: [PATCH 025/111] t12 --- .../connector/CacheDataConnector.scala | 5 +- .../measure/connector/DataConnector.scala | 1 - .../connector/DataConnectorFactory.scala | 5 +- .../connector/DfCacheDataConnector.scala | 136 ++++++--- .../connector/HiveCacheDataConnector.scala | 261 +++++++++++++++++- .../connector/KafkaDataConnector.scala | 79 +++--- .../measure/utils/HdfsFileDumpUtil.scala | 53 ++++ .../src/test/resources/config-streaming.json | 10 +- .../src/test/resources/config-streaming1.json | 63 +++++ measure/src/test/resources/env-streaming.json | 2 +- .../streaming/StreamingAccuracyAlgoTest.scala | 12 +- .../measure/connector/ConnectorTest.scala | 46 +-- 12 files changed, 561 insertions(+), 112 deletions(-) create mode 100644 measure/src/main/scala/org/apache/griffin/measure/utils/HdfsFileDumpUtil.scala create mode 100644 measure/src/test/resources/config-streaming1.json diff --git a/measure/src/main/scala/org/apache/griffin/measure/connector/CacheDataConnector.scala b/measure/src/main/scala/org/apache/griffin/measure/connector/CacheDataConnector.scala index 1c42c7054..52845e81e 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/connector/CacheDataConnector.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/connector/CacheDataConnector.scala @@ -22,6 +22,7 @@ import java.util.concurrent.atomic.AtomicLong import org.apache.griffin.measure.cache.info.{InfoCacheInstance, TimeInfoCache, ZKInfoCache} import org.apache.griffin.measure.utils.TimeUtil +import org.apache.spark.rdd.RDD import org.apache.spark.sql.DataFrame import scala.util.Try @@ -30,9 +31,9 @@ trait CacheDataConnector extends DataConnector { protected val defCacheInfoPath = PathCounter.genPath - def saveData(df: DataFrame, ms: Long): Unit + def saveData(rdd: RDD[Map[String, Any]], ms: Long): Unit - def readData(): Try[DataFrame] + def readData(): Try[RDD[Map[String, Any]]] val cacheInfoPath: String val readyTimeInterval: Long diff --git a/measure/src/main/scala/org/apache/griffin/measure/connector/DataConnector.scala b/measure/src/main/scala/org/apache/griffin/measure/connector/DataConnector.scala index 13f4ab996..86eebe125 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/connector/DataConnector.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/connector/DataConnector.scala @@ -31,7 +31,6 @@ trait DataConnector extends Loggable with Serializable { def cleanOldData(): Unit = {} - def updateOldData(oldDf: DataFrame): Unit = {} def updateOldData(oldRdd: RDD[Map[String, Any]]): Unit = {} } diff --git a/measure/src/main/scala/org/apache/griffin/measure/connector/DataConnectorFactory.scala b/measure/src/main/scala/org/apache/griffin/measure/connector/DataConnectorFactory.scala index add1da3a1..41b90cd57 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/connector/DataConnectorFactory.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/connector/DataConnectorFactory.scala @@ -99,9 +99,8 @@ object DataConnectorFactory { val cacheType = dataCacheParam.cacheType Try { cacheType match { - case DfRegex() => { - DfCacheDataConnector(sqlContext, dataCacheParam) - } + case DfRegex() => DfCacheDataConnector(sqlContext, dataCacheParam) + case HiveRegex() => HiveCacheDataConnector(sqlContext, dataCacheParam) case _ => throw new Exception("cache connector creation error!") } } diff --git a/measure/src/main/scala/org/apache/griffin/measure/connector/DfCacheDataConnector.scala b/measure/src/main/scala/org/apache/griffin/measure/connector/DfCacheDataConnector.scala index 678f56d3c..2d3bf97a1 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/connector/DfCacheDataConnector.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/connector/DfCacheDataConnector.scala @@ -24,8 +24,10 @@ import org.apache.griffin.measure.cache.info.{InfoCacheInstance, TimeInfoCache} import org.apache.griffin.measure.cache.lock.CacheLock import org.apache.griffin.measure.config.params.user.DataCacheParam import org.apache.griffin.measure.result.TimeStampInfo +import org.apache.griffin.measure.rule.DataTypeCalculationUtil import org.apache.griffin.measure.utils.TimeUtil import org.apache.spark.rdd.RDD +import org.apache.spark.sql.types._ import org.apache.spark.sql.{DataFrame, Row, SQLContext} import org.apache.spark.storage.StorageLevel @@ -56,10 +58,9 @@ case class DfCacheDataConnector(sqlContext: SQLContext, dataCacheParam: DataCach } case _ => (0, 0) } - println(deltaTimeRange) val CacheLevel = "cache.level" - val cacheLevel: String = config.getOrElse(CacheLevel, "MEMORY_ONLY").toString + val cacheLevel: String = config.getOrElse(CacheLevel, "MEMORY_AND_DISK").toString val timeStampColumn = TimeStampInfo.key @@ -75,17 +76,19 @@ case class DfCacheDataConnector(sqlContext: SQLContext, dataCacheParam: DataCach true } - def saveData(df: DataFrame, ms: Long): Unit = { + def saveData(rdd: RDD[Map[String, Any]], ms: Long): Unit = { val newCacheLocked = newCacheLock.lock(-1, TimeUnit.SECONDS) if (newCacheLocked) { try { if (newDataFrame == null) { - newDataFrame = df - newDataFrame.persist(StorageLevel.fromString(cacheLevel)) + if (!rdd.isEmpty) { + newDataFrame = genDataFrame(rdd) + newDataFrame.persist(StorageLevel.fromString(cacheLevel)) + } } else { - if (!df.rdd.isEmpty) { + if (!rdd.isEmpty) { newDataFrame.unpersist() - newDataFrame = newDataFrame.unionAll(df) + newDataFrame = newDataFrame.unionAll(genDataFrame(rdd)) newDataFrame.persist(StorageLevel.fromString(cacheLevel)) } } @@ -96,34 +99,9 @@ case class DfCacheDataConnector(sqlContext: SQLContext, dataCacheParam: DataCach newCacheLock.unlock() } } -// if (newDataFrame == null) { -// newDataFrame = df -// newDataFrame.persist(StorageLevel.fromString(cacheLevel)) -// } else { -// if (!df.rdd.isEmpty) { -// newDataFrame.unpersist() -// newDataFrame = newDataFrame.unionAll(df) -// newDataFrame.persist(StorageLevel.fromString(cacheLevel)) -// } -// } -// // submit ms -// submitCacheTime(ms) -// submitReadyTime(ms) } - def readData(): Try[DataFrame] = Try { -// if (initialed) { -// val timeRange = TimeInfoCache.getTimeRange -// println(s"timeRange: ${timeRange}") -// submitLastProcTime(timeRange._2) -// -// val reviseTimeRange = (timeRange._1 + deltaTimeRange._1, timeRange._2 + deltaTimeRange._2) -// println(s"reviseTimeRange: ${reviseTimeRange}") -// dataFrame.filter(s"${timeStampColumn} BETWEEN ${reviseTimeRange._1} AND ${reviseTimeRange._2}") -// } else { -// throw new Exception("data not cached") -// } - + def readData(): Try[RDD[Map[String, Any]]] = Try { val timeRange = TimeInfoCache.getTimeRange println(s"timeRange: ${timeRange}") submitLastProcTime(timeRange._2) @@ -138,6 +116,8 @@ case class DfCacheDataConnector(sqlContext: SQLContext, dataCacheParam: DataCach newDataFrame.unpersist() newDataFrame = newDataFrame.filter(s"${timeStampColumn} > ${reviseTimeRange._2}") tmp + } catch { + case _ => null } finally { newCacheLock.unlock() } @@ -150,14 +130,14 @@ case class DfCacheDataConnector(sqlContext: SQLContext, dataCacheParam: DataCach if (oldDataFrame != null) { oldDataFrame.filter(s"${timeStampColumn} BETWEEN ${reviseTimeRange._1} AND ${reviseTimeRange._2}") } else null + } catch { + case _ => null } finally { oldCacheLock.unlock() } - } else { - throw new Exception("old cache lock unavailable") - } + } else null - if (oldTempDataFrame == null && newTempDataFrame == null) { + val resultDataFrame = if (oldTempDataFrame == null && newTempDataFrame == null) { throw new Exception("data not cached") } else { val finalDataFrame = if (newTempDataFrame == null) { @@ -169,6 +149,11 @@ case class DfCacheDataConnector(sqlContext: SQLContext, dataCacheParam: DataCach } finalDataFrame } + + // data frame -> rdd + resultDataFrame.map { row => + SparkRowFormatter.formatRow(row) + } } override def cleanOldData(): Unit = { @@ -202,18 +187,23 @@ case class DfCacheDataConnector(sqlContext: SQLContext, dataCacheParam: DataCach // } } - override def updateOldData(oldDf: DataFrame): Unit = { + override def updateOldData(oldRdd: RDD[Map[String, Any]]): Unit = { val oldCacheLocked = oldCacheLock.lock(-1, TimeUnit.SECONDS) if (oldCacheLocked) { try { if (oldDataFrame == null) { - oldDataFrame = oldDf - oldDataFrame.persist(StorageLevel.fromString(cacheLevel)) + if (!oldRdd.isEmpty) { + oldDataFrame = genDataFrame(oldRdd) + oldDataFrame.persist(StorageLevel.fromString(cacheLevel)) + } } else { - if (!oldDf.rdd.isEmpty) { + if (!oldRdd.isEmpty) { oldDataFrame.unpersist() - oldDataFrame = oldDf + oldDataFrame = genDataFrame(oldRdd) oldDataFrame.persist(StorageLevel.fromString(cacheLevel)) + } else { + oldDataFrame.unpersist() + oldDataFrame = null } } } finally { @@ -222,4 +212,66 @@ case class DfCacheDataConnector(sqlContext: SQLContext, dataCacheParam: DataCach } } + // generate DataFrame + // maybe we can directly use def createDataFrame[A <: Product : TypeTag](rdd: RDD[A]): DataFrame + // to avoid generate data type by myself, just translate each value into Product + private def genDataFrame(rdd: RDD[Map[String, Any]]): DataFrame = { + val fields = rdd.aggregate(Map[String, DataType]())( + DataTypeCalculationUtil.sequenceDataTypeMap, DataTypeCalculationUtil.combineDataTypeMap + ).toList.map(f => StructField(f._1, f._2)) + val schema = StructType(fields) + val datas: RDD[Row] = rdd.map { d => + val values = fields.map { field => + val StructField(k, dt, _, _) = field + d.get(k) match { + case Some(v) => v + case _ => null + } + } + Row(values: _*) + } + val df = sqlContext.createDataFrame(datas, schema) + df + } + +} + +import scala.collection.mutable.{ArrayBuffer} + +object SparkRowFormatter { + + def formatRow(row: Row): Map[String, Any] = { + formatRowWithSchema(row, row.schema) + } + + private def formatRowWithSchema(row: Row, schema: StructType): Map[String, Any] = { + formatStruct(schema.fields, row) + } + + private def formatStruct(schema: Seq[StructField], r: Row) = { + val paired = schema.zip(r.toSeq) + paired.foldLeft(Map[String, Any]())((s, p) => s ++ formatItem(p)) + } + + private def formatItem(p: Pair[StructField, Any]): Map[String, Any] = { + p match { + case (sf, a) => + sf.dataType match { + case ArrayType(et, _) => + Map(sf.name -> (if (a == null) a else formatArray(et, a.asInstanceOf[ArrayBuffer[Any]]))) + case StructType(s) => + Map(sf.name -> (if (a == null) a else formatStruct(s, a.asInstanceOf[Row]))) + case _ => Map(sf.name -> a) + } + } + } + + private def formatArray(et: DataType, arr: ArrayBuffer[Any]): Seq[Any] = { + et match { + case StructType(s) => arr.map(e => formatStruct(s, e.asInstanceOf[Row])) + case ArrayType(t, _) => + arr.map(e => formatArray(t, e.asInstanceOf[ArrayBuffer[Any]])) + case _ => arr + } + } } \ No newline at end of file diff --git a/measure/src/main/scala/org/apache/griffin/measure/connector/HiveCacheDataConnector.scala b/measure/src/main/scala/org/apache/griffin/measure/connector/HiveCacheDataConnector.scala index 6db66778c..cdb0e951b 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/connector/HiveCacheDataConnector.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/connector/HiveCacheDataConnector.scala @@ -18,9 +18,18 @@ under the License. */ package org.apache.griffin.measure.connector +import java.util.concurrent.TimeUnit + +import org.apache.griffin.measure.cache.info.{InfoCacheInstance, TimeInfoCache} import org.apache.griffin.measure.config.params.user.DataCacheParam -import org.apache.spark.sql.SQLContext +import org.apache.griffin.measure.result.TimeStampInfo +import org.apache.griffin.measure.utils.TimeUtil +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.{DataFrame, SQLContext} import org.apache.spark.sql.hive.HiveContext +import org.apache.spark.storage.StorageLevel + +import scala.util.Try case class HiveCacheDataConnector(sqlContext: SQLContext, dataCacheParam: DataCacheParam ) extends CacheDataConnector { @@ -29,4 +38,254 @@ case class HiveCacheDataConnector(sqlContext: SQLContext, dataCacheParam: DataCa throw new Exception("hive context not prepared!") } + val config = dataCacheParam.config + val InfoPath = "info.path" + val cacheInfoPath: String = config.getOrElse(InfoPath, defCacheInfoPath).toString + + val newCacheLock = InfoCacheInstance.genLock(s"${cacheInfoPath}.new") + val oldCacheLock = InfoCacheInstance.genLock(s"${cacheInfoPath}.old") + + val timeRangeParam: List[String] = if (dataCacheParam.timeRange != null) dataCacheParam.timeRange else Nil + val deltaTimeRange: (Long, Long) = (timeRangeParam ::: List("0", "0")) match { + case s :: e :: _ => { + val ns = TimeUtil.milliseconds(s) match { + case Some(n) if (n < 0) => n + case _ => 0 + } + val ne = TimeUtil.milliseconds(e) match { + case Some(n) if (n < 0) => n + case _ => 0 + } + (ns, ne) + } + case _ => (0, 0) + } + + val TableName = "table.name" + val tableName: String = config.get(TableName) match { + case Some(s: String) => s + case _ => throw new Exception("invalid table.name!") + } + val ParentPath = "parent.path" + val parentPath: String = config.get(ParentPath) match { + case Some(s: String) => s + case _ => throw new Exception("invalid parent.path!") + } + val tablePath = getFilePath(parentPath, tableName) + + val timeStampColumn = TimeStampInfo.key + + val ReadyTimeInterval = "ready.time.interval" + val ReadyTimeDelay = "ready.time.delay" + val readyTimeInterval: Long = TimeUtil.milliseconds(config.getOrElse(ReadyTimeInterval, "1m").toString).getOrElse(60000L) + val readyTimeDelay: Long = TimeUtil.milliseconds(config.getOrElse(ReadyTimeDelay, "1m").toString).getOrElse(60000L) + + type Schema = (Long, String) + val schema: List[(String, String)] = List( + ("tmst", "bigint"), + ("payload", "string") + ) + type Partition = (Long, Long) + val partition: List[(String, String, String)] = List( + ("hr", "bigint", "hour"), + ("min", "bigint", "min") + ) + + private val fieldSep = """,""" + private val rowSep = """\n""" + + protected def getFilePath(parentPath: String, fileName: String): String = { + if (parentPath.endsWith("/")) parentPath + fileName else parentPath + "/" + fileName + } + + override def init(): Unit = { + val colsSql = schema.map { field => + s"`${field._1}` ${field._2}" + }.mkString(", ") + val partitionsSql = partition.map { partition => + s"`${partition._1}` ${partition._2}" + } + val sql = s"""CREATE EXTERNAL TABLE IF NOT EXISTS `${tableName}` + |(${colsSql}) PARTITIONED BY (${partitionsSql}) + |ROW FORMAT DELIMITED + |FIELDS TERMINATED BY '${fieldSep}' + |LINES TERMINATED BY '${rowSep}' + |STORED AS TEXTFILE + |LOCATION '${tablePath}'""".stripMargin + sqlContext.sql(sql) + } + + def available(): Boolean = { + true + } + + private def encode(data: Map[String, Any]): Schema = { + ; + } + + private def decode(data: Schema): Map[String, Any] = { + ; + } + + private def getPartition(ms: Long): List[(String, Any)] = { + partition.map { p => + val (name, _, unit) = p + val t = TimeUtil.timeToUnit(ms, unit) + (name, t) + } + } + private def genPartitionHdfsPath(partitions: List[(String, Any)]): String = { + partitions.map(prtn => s"${prtn._1}=${prtn._2}").mkString("/") + } + + def saveData(rdd: RDD[Map[String, Any]], ms: Long): Unit = { + val newCacheLocked = newCacheLock.lock(-1, TimeUnit.SECONDS) + if (newCacheLocked) { + try { + val ptns = getPartition(ms) + val ptnsSql = ptns.map(ptn => (s"`${ptn._1}`=${ptn._2}")).mkString(", ") + val ptnsPath = genPartitionHdfsPath(ptns) + val filePath = s"${tablePath}/${ptnsPath}/${ms}" + + // encode data + val dataRdd = rdd.map(encode(_)) + + // save data to hdfs + // fixme: waiting... + + // add partition info + + // submit ms + submitCacheTime(ms) + submitReadyTime(ms) + } finally { + newCacheLock.unlock() + } + } + // if (newDataFrame == null) { + // newDataFrame = df + // newDataFrame.persist(StorageLevel.fromString(cacheLevel)) + // } else { + // if (!df.rdd.isEmpty) { + // newDataFrame.unpersist() + // newDataFrame = newDataFrame.unionAll(df) + // newDataFrame.persist(StorageLevel.fromString(cacheLevel)) + // } + // } + // // submit ms + // submitCacheTime(ms) + // submitReadyTime(ms) + } + + def readData(): Try[RDD[Map[String, Any]]] = Try { + // if (initialed) { + // val timeRange = TimeInfoCache.getTimeRange + // println(s"timeRange: ${timeRange}") + // submitLastProcTime(timeRange._2) + // + // val reviseTimeRange = (timeRange._1 + deltaTimeRange._1, timeRange._2 + deltaTimeRange._2) + // println(s"reviseTimeRange: ${reviseTimeRange}") + // dataFrame.filter(s"${timeStampColumn} BETWEEN ${reviseTimeRange._1} AND ${reviseTimeRange._2}") + // } else { + // throw new Exception("data not cached") + // } + + val timeRange = TimeInfoCache.getTimeRange + println(s"timeRange: ${timeRange}") + submitLastProcTime(timeRange._2) + val reviseTimeRange = (timeRange._1 + deltaTimeRange._1, timeRange._2 + deltaTimeRange._2) + println(s"reviseTimeRange: ${reviseTimeRange}") + + // move new data frame to temp data frame + val newCacheLocked = newCacheLock.lock(-1, TimeUnit.SECONDS) + val newTempDataFrame = if (newCacheLocked) { + try { + val tmp = newDataFrame.filter(s"${timeStampColumn} BETWEEN ${reviseTimeRange._1} AND ${reviseTimeRange._2}") + newDataFrame.unpersist() + newDataFrame = newDataFrame.filter(s"${timeStampColumn} > ${reviseTimeRange._2}") + tmp + } finally { + newCacheLock.unlock() + } + } else null + + // add temp data frame to old data frame + val oldCacheLocked = oldCacheLock.lock(-1, TimeUnit.SECONDS) + val oldTempDataFrame = if (oldCacheLocked) { + try { + if (oldDataFrame != null) { + oldDataFrame.filter(s"${timeStampColumn} BETWEEN ${reviseTimeRange._1} AND ${reviseTimeRange._2}") + } else null + } finally { + oldCacheLock.unlock() + } + } else { + throw new Exception("old cache lock unavailable") + } + + if (oldTempDataFrame == null && newTempDataFrame == null) { + throw new Exception("data not cached") + } else { + val finalDataFrame = if (newTempDataFrame == null) { + oldTempDataFrame + } else if (oldTempDataFrame == null) { + newTempDataFrame + } else { + oldTempDataFrame.unionAll(newTempDataFrame) + } + finalDataFrame + } + } + + override def cleanOldData(): Unit = { + val oldCacheLocked = oldCacheLock.lock(-1, TimeUnit.SECONDS) + if (oldCacheLocked) { + try { + val timeRange = TimeInfoCache.getTimeRange + val reviseTimeRange = (timeRange._1 + deltaTimeRange._1, timeRange._2 + deltaTimeRange._2) + println(s"clean reviseTimeRange: ${reviseTimeRange}") + + oldDataFrame.unpersist() + oldDataFrame = oldDataFrame.filter(s"${timeStampColumn} >= ${reviseTimeRange._1}") + oldDataFrame.persist(StorageLevel.fromString(cacheLevel)) + } finally { + oldCacheLock.unlock() + } + } + + // if (initialed) { + // val timeRange = TimeInfoCache.getTimeRange + // val reviseTimeRange = (timeRange._1 + deltaTimeRange._1, timeRange._2 + deltaTimeRange._2) + // println(s"clean reviseTimeRange: ${reviseTimeRange}") + // + // dataFrame.show(10) + // + // dataFrame.unpersist() + // dataFrame = dataFrame.filter(s"${timeStampColumn} >= ${reviseTimeRange._1}") + // dataFrame.persist(StorageLevel.fromString(cacheLevel)) + // + // dataFrame.show(10) + // } + } + + override def updateOldData(oldRdd: RDD[Map[String, Any]]): Unit = { + val oldCacheLocked = oldCacheLock.lock(-1, TimeUnit.SECONDS) + if (oldCacheLocked) { + try { + if (oldDataFrame == null) { + oldDataFrame = oldDf + oldDataFrame.persist(StorageLevel.fromString(cacheLevel)) + } else { + if (!oldDf.rdd.isEmpty) { + oldDataFrame.unpersist() + oldDataFrame = oldDf + oldDataFrame.persist(StorageLevel.fromString(cacheLevel)) + } + } + } finally { + oldCacheLock.unlock() + } + } + } + } diff --git a/measure/src/main/scala/org/apache/griffin/measure/connector/KafkaDataConnector.scala b/measure/src/main/scala/org/apache/griffin/measure/connector/KafkaDataConnector.scala index b3c70cc6f..a624ccc8e 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/connector/KafkaDataConnector.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/connector/KafkaDataConnector.scala @@ -81,6 +81,8 @@ case class KafkaDataConnector(sqlContext: SQLContext, @transient ssc: StreamingC } override def init(): Unit = { + cacheDataConnector.init + val ds = kafkaStreamingDataConnector.stream match { case Success(dstream) => dstream case Failure(ex) => throw ex @@ -104,34 +106,34 @@ case class KafkaDataConnector(sqlContext: SQLContext, @transient ssc: StreamingC } // generate DataFrame - val df = genDataFrame(valueMapRdd) +// val df = genDataFrame(valueMapRdd) // save data frame - cacheDataConnector.saveData(df, ms) + cacheDataConnector.saveData(valueMapRdd, ms) }) } - // generate DataFrame - // maybe we can directly use def createDataFrame[A <: Product : TypeTag](rdd: RDD[A]): DataFrame - // to avoid generate data type by myself, just translate each value into Product - private def genDataFrame(rdd: RDD[Map[String, Any]]): DataFrame = { - val fields = rdd.aggregate(Map[String, DataType]())( - DataTypeCalculationUtil.sequenceDataTypeMap, DataTypeCalculationUtil.combineDataTypeMap - ).toList.map(f => StructField(f._1, f._2)) - val schema = StructType(fields) - val datas: RDD[Row] = rdd.map { d => - val values = fields.map { field => - val StructField(k, dt, _, _) = field - d.get(k) match { - case Some(v) => v - case _ => null - } - } - Row(values: _*) - } - val df = sqlContext.createDataFrame(datas, schema) - df - } +// // generate DataFrame +// // maybe we can directly use def createDataFrame[A <: Product : TypeTag](rdd: RDD[A]): DataFrame +// // to avoid generate data type by myself, just translate each value into Product +// private def genDataFrame(rdd: RDD[Map[String, Any]]): DataFrame = { +// val fields = rdd.aggregate(Map[String, DataType]())( +// DataTypeCalculationUtil.sequenceDataTypeMap, DataTypeCalculationUtil.combineDataTypeMap +// ).toList.map(f => StructField(f._1, f._2)) +// val schema = StructType(fields) +// val datas: RDD[Row] = rdd.map { d => +// val values = fields.map { field => +// val StructField(k, dt, _, _) = field +// d.get(k) match { +// case Some(v) => v +// case _ => null +// } +// } +// Row(values: _*) +// } +// val df = sqlContext.createDataFrame(datas, schema) +// df +// } def metaData(): Try[Iterable[(String, String)]] = Try { Map.empty[String, String] @@ -139,21 +141,32 @@ case class KafkaDataConnector(sqlContext: SQLContext, @transient ssc: StreamingC def data(): Try[RDD[(Product, (Map[String, Any], Map[String, Any]))]] = Try { cacheDataConnector.readData match { - case Success(df) => { - df.flatMap { row => + case Success(rdd) => { + rdd.flatMap { row => // generate cache data // val cacheExprValueMaps = ExprValueUtil.genExprValueMaps(Some(row), ruleExprs.cacheExprs, constFinalExprValueMap) // val finalExprValueMaps = ExprValueUtil.updateExprValueMaps(ruleExprs.finalCacheExprs, cacheExprValueMaps) - val finalExprValueMap = ruleExprs.finalCacheExprs.foldLeft(Map[String, Any]()) { (mp, expr) => - mp + (expr._id -> row.getAs[Any](expr._id)) - } +// val finalExprValueMap = ruleExprs.finalCacheExprs.foldLeft(Map[String, Any]()) { (mp, expr) => +// mp + (expr._id -> row.get(expr._id)) +// } + val finalExprValueMap = ruleExprs.finalCacheExprs.flatMap { expr => + row.get(expr._id).flatMap { d => + Some((expr._id, d)) + } + }.toMap // data info +// val dataInfoMap: Map[String, Any] = DataInfo.cacheInfoList.map { info => +// try { +// (info.key -> row.getAs[info.T](info.key)) +// } catch { +// case e: Throwable => info.defWrap +// } +// }.toMap val dataInfoMap: Map[String, Any] = DataInfo.cacheInfoList.map { info => - try { - (info.key -> row.getAs[info.T](info.key)) - } catch { - case e: Throwable => info.defWrap + row.get(info.key) match { + case Some(d) => (info.key -> d) + case _ => info.defWrap } }.toMap @@ -178,7 +191,7 @@ case class KafkaDataConnector(sqlContext: SQLContext, @transient ssc: StreamingC override def updateOldData(oldRdd: RDD[Map[String, Any]]): Unit = { if (dataConnectorParam.getMatchOnce) { - cacheDataConnector.updateOldData(genDataFrame(oldRdd)) + cacheDataConnector.updateOldData(oldRdd) } } diff --git a/measure/src/main/scala/org/apache/griffin/measure/utils/HdfsFileDumpUtil.scala b/measure/src/main/scala/org/apache/griffin/measure/utils/HdfsFileDumpUtil.scala new file mode 100644 index 000000000..ede6e39ab --- /dev/null +++ b/measure/src/main/scala/org/apache/griffin/measure/utils/HdfsFileDumpUtil.scala @@ -0,0 +1,53 @@ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ +package org.apache.griffin.measure.utils + +import org.apache.spark.rdd.RDD + +object HdfsFileDumpUtil { + + val sepCount = 5000 + + private def suffix(i: Long): String = { + if (i == 0) "" else s".${i}" + } + + def splitRdd[T](rdd: RDD[T])(implicit m: Manifest[T]): RDD[(Long, Iterable[T])] = { + val indexRdd = rdd.zipWithIndex + indexRdd.map(p => ((p._2 / sepCount), p._1)).groupByKey() + } + + private def directDump(path: String, list: Iterable[String], lineSep: String): Unit = { + // collect and save + val strRecords = list.mkString(lineSep) + // save into hdfs + HdfsUtil.writeContent(path, strRecords) + } + + def dump(path: String, recordsRdd: RDD[String], lineSep: String): Boolean = { + val groupedRdd = splitRdd(recordsRdd) + groupedRdd.aggregate(true)({ (res, pair) => + val (idx, list) = pair + val filePath = path + suffix(idx) + directDump(filePath, list, lineSep) + true + }, _ && _) + } + +} diff --git a/measure/src/test/resources/config-streaming.json b/measure/src/test/resources/config-streaming.json index d827f0f7b..93c410feb 100644 --- a/measure/src/test/resources/config-streaming.json +++ b/measure/src/test/resources/config-streaming.json @@ -17,9 +17,10 @@ "value.type": "java.lang.String" }, "cache": { - "type": "df", + "type": "hive", "config": { - "cache.level": "MEMORY_AND_DISK", + "table.name": "src", + "parent.path": "hdfs://localhost/griffin/streaming/dump", "info.path": "src", "ready.time.interval": "10s", "ready.time.delay": "0" @@ -44,9 +45,10 @@ "value.type": "java.lang.String" }, "cache": { - "type": "df", + "type": "hive", "config": { - "cache.level": "MEMORY_AND_DISK", + "table.name": "tgt", + "parent.path": "hdfs://localhost/griffin/streaming/dump", "info.path": "tgt", "ready.time.interval": "10s", "ready.time.delay": "0" diff --git a/measure/src/test/resources/config-streaming1.json b/measure/src/test/resources/config-streaming1.json new file mode 100644 index 000000000..d827f0f7b --- /dev/null +++ b/measure/src/test/resources/config-streaming1.json @@ -0,0 +1,63 @@ +{ + "name": "accu2", + "type": "accuracy", + + "source": { + "type": "kafka", + "version": "0.8", + "config": { + "kafka.config": { + "bootstrap.servers": "10.149.247.156:9092", + "group.id": "group1", + "auto.offset.reset": "smallest", + "auto.commit.enable": "false" + }, + "topics": "sss", + "key.type": "java.lang.String", + "value.type": "java.lang.String" + }, + "cache": { + "type": "df", + "config": { + "cache.level": "MEMORY_AND_DISK", + "info.path": "src", + "ready.time.interval": "10s", + "ready.time.delay": "0" + }, + "time.range": ["-2m", "0"] + }, + "match.once": true + }, + + "target": { + "type": "kafka", + "version": "0.8", + "config": { + "kafka.config": { + "bootstrap.servers": "10.149.247.156:9092", + "group.id": "group1", + "auto.offset.reset": "smallest", + "auto.commit.enable": "false" + }, + "topics": "ttt", + "key.type": "java.lang.String", + "value.type": "java.lang.String" + }, + "cache": { + "type": "df", + "config": { + "cache.level": "MEMORY_AND_DISK", + "info.path": "tgt", + "ready.time.interval": "10s", + "ready.time.delay": "0" + }, + "time.range": ["-2m", "0"] + }, + "match.once": false + }, + + "evaluateRule": { + "sampleRatio": 0.2, + "rules": "$source.json().name = $target.json().name AND $source.json().age = $target.json().age" + } +} \ No newline at end of file diff --git a/measure/src/test/resources/env-streaming.json b/measure/src/test/resources/env-streaming.json index 761a4a527..c807080b6 100644 --- a/measure/src/test/resources/env-streaming.json +++ b/measure/src/test/resources/env-streaming.json @@ -1,7 +1,7 @@ { "spark": { "log.level": "WARN", - "checkpoint.dir": "/test/griffin/cp", + "checkpoint.dir": "hdfs://localhost/test/griffin/cp", "batch.interval": "2s", "process.interval": "10s", "config": {} diff --git a/measure/src/test/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgoTest.scala b/measure/src/test/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgoTest.scala index 866534c24..090c6f58a 100644 --- a/measure/src/test/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgoTest.scala +++ b/measure/src/test/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgoTest.scala @@ -38,6 +38,7 @@ import org.apache.griffin.measure.rule.{ExprValueUtil, RuleAnalyzer, RuleFactory import org.apache.griffin.measure.utils.TimeUtil import org.apache.spark.rdd.RDD import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.hive.HiveContext import org.apache.spark.streaming.{Milliseconds, StreamingContext} import org.apache.spark.{SparkConf, SparkContext} import org.junit.runner.RunWith @@ -51,7 +52,7 @@ import scala.util.{Failure, Success, Try} class StreamingAccuracyAlgoTest extends FunSuite with Matchers with BeforeAndAfter with Loggable { val envFile = "src/test/resources/env-streaming.json" - val confFile = "src/test/resources/config-streaming.json" + val confFile = "src/test/resources/config-streaming1.json" val envFsType = "local" val userFsType = "local" @@ -96,7 +97,12 @@ class StreamingAccuracyAlgoTest extends FunSuite with Matchers with BeforeAndAft val conf = new SparkConf().setMaster("local[*]").setAppName(metricName) sc = new SparkContext(conf) sc.setLogLevel(envParam.sparkParam.logLevel) - sqlContext = new SQLContext(sc) +// sqlContext = new SQLContext(sc) + sqlContext = new HiveContext(sc) + + val a = sqlContext.sql("select * from test limit 10") + // val a = sqlContext.sql("show tables") + a.show(10) } test("algorithm") { @@ -278,6 +284,8 @@ class StreamingAccuracyAlgoTest extends FunSuite with Matchers with BeforeAndAft val et = new Date().getTime appPersist.log(et, s"persist using time: ${et - ct} ms") + } catch { + case e: Throwable => error(s"process error: ${e.getMessage}") } finally { lock.unlock() } diff --git a/measure/src/test/scala/org/apache/griffin/measure/connector/ConnectorTest.scala b/measure/src/test/scala/org/apache/griffin/measure/connector/ConnectorTest.scala index 64b5c0743..388f5d35b 100644 --- a/measure/src/test/scala/org/apache/griffin/measure/connector/ConnectorTest.scala +++ b/measure/src/test/scala/org/apache/griffin/measure/connector/ConnectorTest.scala @@ -161,24 +161,24 @@ class ConnectorTest extends FunSuite with Matchers with BeforeAndAfter { /// - def genDataFrame(rdd: RDD[Map[String, Any]]): DataFrame = { - val fields = rdd.aggregate(Map[String, DataType]())( - DataTypeCalculationUtil.sequenceDataTypeMap, DataTypeCalculationUtil.combineDataTypeMap - ).toList.map(f => StructField(f._1, f._2)) - val schema = StructType(fields) - val datas: RDD[Row] = rdd.map { d => - val values = fields.map { field => - val StructField(k, dt, _, _) = field - d.get(k) match { - case Some(v) => v - case _ => null - } - } - Row(values: _*) - } - val df = sqlContext.createDataFrame(datas, schema) - df - } +// def genDataFrame(rdd: RDD[Map[String, Any]]): DataFrame = { +// val fields = rdd.aggregate(Map[String, DataType]())( +// DataTypeCalculationUtil.sequenceDataTypeMap, DataTypeCalculationUtil.combineDataTypeMap +// ).toList.map(f => StructField(f._1, f._2)) +// val schema = StructType(fields) +// val datas: RDD[Row] = rdd.map { d => +// val values = fields.map { field => +// val StructField(k, dt, _, _) = field +// d.get(k) match { +// case Some(v) => v +// case _ => null +// } +// } +// Row(values: _*) +// } +// val df = sqlContext.createDataFrame(datas, schema) +// df +// } val rules = "$source.json().name = 's2' AND $source.json().age = 32" val ep = EvaluateRuleParam(1, rules) @@ -230,11 +230,11 @@ class ConnectorTest extends FunSuite with Matchers with BeforeAndAfter { // println(s"count: ${cnt}\n${valuestr}") // generate DataFrame - val df = genDataFrame(valueMapRdd) +// val df = genDataFrame(valueMapRdd) // df.show(10) // save data frame - cacheDataConnector.saveData(df, ms) + cacheDataConnector.saveData(valueMapRdd, ms) // show data // cacheDataConnector.readData() match { @@ -254,9 +254,9 @@ class ConnectorTest extends FunSuite with Matchers with BeforeAndAfter { try { // show data cacheDataConnector.readData() match { - case Success(rdf) => { - rdf.show(10) - println(s"count: ${rdf.count}") + case Success(rdd) => { + rdd.take(10).foreach(println) + println(s"count: ${rdd.count}") } case Failure(ex) => println(s"cache data error: ${ex.getMessage}") } From d8e6b66686f55b126faea74595d0c14ee31b5d14 Mon Sep 17 00:00:00 2001 From: Lionel Liu Date: Fri, 21 Jul 2017 10:57:36 +0800 Subject: [PATCH 026/111] t13 --- .../measure/cache/info/TimeInfoCache.scala | 2 + .../connector/CacheDataConnector.scala | 18 + .../measure/connector/DataConnector.scala | 3 +- .../connector/DfCacheDataConnector.scala | 8 +- .../connector/HiveCacheDataConnector.scala | 336 +++++++++++------- .../connector/KafkaDataConnector.scala | 10 +- .../griffin/measure/persist/HdfsPersist.scala | 6 +- .../measure/utils/HdfsFileDumpUtil.scala | 30 ++ .../griffin/measure/utils/HdfsUtil.scala | 13 + .../griffin/measure/utils/JsonUtil.scala | 2 +- .../src/test/resources/config-streaming.json | 2 + .../streaming/StreamingAccuracyAlgoTest.scala | 83 +++-- .../measure/connector/ConnectorTest.scala | 5 +- 13 files changed, 335 insertions(+), 183 deletions(-) diff --git a/measure/src/main/scala/org/apache/griffin/measure/cache/info/TimeInfoCache.scala b/measure/src/main/scala/org/apache/griffin/measure/cache/info/TimeInfoCache.scala index cdd179853..0526e8011 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/cache/info/TimeInfoCache.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/cache/info/TimeInfoCache.scala @@ -25,10 +25,12 @@ object TimeInfoCache extends Loggable with Serializable { private val CacheTime = "cache.time" private val LastProcTime = "last.proc.time" private val ReadyTime = "ready.time" + private val CleanTime = "clean.time" def cacheTime(path: String): String = s"${path}/${CacheTime}" def lastProcTime(path: String): String = s"${path}/${LastProcTime}" def readyTime(path: String): String = s"${path}/${ReadyTime}" + def cleanTime(path: String): String = s"${path}/${CleanTime}" val infoPath = "info" diff --git a/measure/src/main/scala/org/apache/griffin/measure/connector/CacheDataConnector.scala b/measure/src/main/scala/org/apache/griffin/measure/connector/CacheDataConnector.scala index 52845e81e..855d606b8 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/connector/CacheDataConnector.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/connector/CacheDataConnector.scala @@ -44,6 +44,7 @@ trait CacheDataConnector extends DataConnector { def selfCacheTime = TimeInfoCache.cacheTime(selfCacheInfoPath) def selfLastProcTime = TimeInfoCache.lastProcTime(selfCacheInfoPath) def selfReadyTime = TimeInfoCache.readyTime(selfCacheInfoPath) + def selfCleanTime = TimeInfoCache.cleanTime(selfCacheInfoPath) protected def submitCacheTime(ms: Long): Unit = { val map = Map[String, String]((selfCacheTime -> ms.toString)) @@ -63,6 +64,23 @@ trait CacheDataConnector extends DataConnector { InfoCacheInstance.cacheInfo(map) } + protected def submitCleanTime(ms: Long): Unit = { + val map = Map[String, String]((selfCleanTime -> ms.toString)) + InfoCacheInstance.cacheInfo(map) + } + + protected def readCleanTime(): Option[Long] = { + val key = selfCleanTime + val keys = key :: Nil + InfoCacheInstance.readInfo(keys).get(key).flatMap { v => + try { + Some(v.toLong) + } catch { + case _ => None + } + } + } + } object PathCounter { diff --git a/measure/src/main/scala/org/apache/griffin/measure/connector/DataConnector.scala b/measure/src/main/scala/org/apache/griffin/measure/connector/DataConnector.scala index 86eebe125..527676be7 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/connector/DataConnector.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/connector/DataConnector.scala @@ -31,6 +31,7 @@ trait DataConnector extends Loggable with Serializable { def cleanOldData(): Unit = {} - def updateOldData(oldRdd: RDD[Map[String, Any]]): Unit = {} + def updateOldData(t: Long, oldData: Iterable[Map[String, Any]]): Unit = {} + def updateAllOldData(oldRdd: RDD[Map[String, Any]]): Unit = {} } diff --git a/measure/src/main/scala/org/apache/griffin/measure/connector/DfCacheDataConnector.scala b/measure/src/main/scala/org/apache/griffin/measure/connector/DfCacheDataConnector.scala index 2d3bf97a1..f2e49bacf 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/connector/DfCacheDataConnector.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/connector/DfCacheDataConnector.scala @@ -95,6 +95,8 @@ case class DfCacheDataConnector(sqlContext: SQLContext, dataCacheParam: DataCach // submit ms submitCacheTime(ms) submitReadyTime(ms) + } catch { + case e: Throwable => error(s"save data error: ${e.getMessage}") } finally { newCacheLock.unlock() } @@ -167,6 +169,8 @@ case class DfCacheDataConnector(sqlContext: SQLContext, dataCacheParam: DataCach oldDataFrame.unpersist() oldDataFrame = oldDataFrame.filter(s"${timeStampColumn} >= ${reviseTimeRange._1}") oldDataFrame.persist(StorageLevel.fromString(cacheLevel)) + } catch { + case e: Throwable => error(s"clean old data error: ${e.getMessage}") } finally { oldCacheLock.unlock() } @@ -187,7 +191,7 @@ case class DfCacheDataConnector(sqlContext: SQLContext, dataCacheParam: DataCach // } } - override def updateOldData(oldRdd: RDD[Map[String, Any]]): Unit = { + override def updateAllOldData(oldRdd: RDD[Map[String, Any]]): Unit = { val oldCacheLocked = oldCacheLock.lock(-1, TimeUnit.SECONDS) if (oldCacheLocked) { try { @@ -206,6 +210,8 @@ case class DfCacheDataConnector(sqlContext: SQLContext, dataCacheParam: DataCach oldDataFrame = null } } + } catch { + case e: Throwable => error(s"update all old data error: ${e.getMessage}") } finally { oldCacheLock.unlock() } diff --git a/measure/src/main/scala/org/apache/griffin/measure/connector/HiveCacheDataConnector.scala b/measure/src/main/scala/org/apache/griffin/measure/connector/HiveCacheDataConnector.scala index cdb0e951b..6b83c0039 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/connector/HiveCacheDataConnector.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/connector/HiveCacheDataConnector.scala @@ -23,13 +23,13 @@ import java.util.concurrent.TimeUnit import org.apache.griffin.measure.cache.info.{InfoCacheInstance, TimeInfoCache} import org.apache.griffin.measure.config.params.user.DataCacheParam import org.apache.griffin.measure.result.TimeStampInfo -import org.apache.griffin.measure.utils.TimeUtil +import org.apache.griffin.measure.utils.{HdfsFileDumpUtil, HdfsUtil, JsonUtil, TimeUtil} import org.apache.spark.rdd.RDD import org.apache.spark.sql.{DataFrame, SQLContext} import org.apache.spark.sql.hive.HiveContext import org.apache.spark.storage.StorageLevel -import scala.util.Try +import scala.util.{Success, Try} case class HiveCacheDataConnector(sqlContext: SQLContext, dataCacheParam: DataCacheParam ) extends CacheDataConnector { @@ -61,9 +61,11 @@ case class HiveCacheDataConnector(sqlContext: SQLContext, dataCacheParam: DataCa case _ => (0, 0) } + val Database = "database" + val database: String = config.getOrElse(Database, "").toString val TableName = "table.name" val tableName: String = config.get(TableName) match { - case Some(s: String) => s + case Some(s: String) if (s.nonEmpty) => s case _ => throw new Exception("invalid table.name!") } val ParentPath = "parent.path" @@ -71,169 +73,178 @@ case class HiveCacheDataConnector(sqlContext: SQLContext, dataCacheParam: DataCa case Some(s: String) => s case _ => throw new Exception("invalid parent.path!") } - val tablePath = getFilePath(parentPath, tableName) + val tablePath = HdfsUtil.getHdfsFilePath(parentPath, tableName) - val timeStampColumn = TimeStampInfo.key + val concreteTableName = if (dbPrefix) s"${database}.${tableName}" else tableName val ReadyTimeInterval = "ready.time.interval" val ReadyTimeDelay = "ready.time.delay" val readyTimeInterval: Long = TimeUtil.milliseconds(config.getOrElse(ReadyTimeInterval, "1m").toString).getOrElse(60000L) val readyTimeDelay: Long = TimeUtil.milliseconds(config.getOrElse(ReadyTimeDelay, "1m").toString).getOrElse(60000L) - type Schema = (Long, String) + val TimeStampColumn: String = TimeStampInfo.key + val PayloadColumn: String = "payload" + +// type Schema = (Long, String) val schema: List[(String, String)] = List( - ("tmst", "bigint"), - ("payload", "string") + (TimeStampColumn, "bigint"), + (PayloadColumn, "string") ) - type Partition = (Long, Long) + val schemaName = schema.map(_._1) + +// type Partition = (Long, Long) val partition: List[(String, String, String)] = List( ("hr", "bigint", "hour"), ("min", "bigint", "min") ) + val partitionName = partition.map(_._1) - private val fieldSep = """,""" + private val fieldSep = """|""" private val rowSep = """\n""" + private val rowSepLiteral = "\n" - protected def getFilePath(parentPath: String, fileName: String): String = { - if (parentPath.endsWith("/")) parentPath + fileName else parentPath + "/" + fileName + private def dbPrefix(): Boolean = { + database.nonEmpty && !database.equals("default") + } + + private def tableExists(): Boolean = { + Try { + if (dbPrefix) { + sqlContext.tables(database).filter(tableExistsSql).collect.size + } else { + sqlContext.tables().filter(tableExistsSql).collect.size + } + } match { + case Success(s) => s > 0 + case _ => false + } } override def init(): Unit = { - val colsSql = schema.map { field => - s"`${field._1}` ${field._2}" - }.mkString(", ") - val partitionsSql = partition.map { partition => - s"`${partition._1}` ${partition._2}" + try { + if (tableExists) { + // drop exist table + val dropSql = s"""DROP TABLE ${concreteTableName}""" + sqlContext.sql(dropSql) + } + + val colsSql = schema.map { field => + s"`${field._1}` ${field._2}" + }.mkString(", ") + val partitionsSql = partition.map { partition => + s"`${partition._1}` ${partition._2}" + }.mkString(", ") + val sql = s"""CREATE EXTERNAL TABLE IF NOT EXISTS ${concreteTableName} + |(${colsSql}) PARTITIONED BY (${partitionsSql}) + |ROW FORMAT DELIMITED + |FIELDS TERMINATED BY '${fieldSep}' + |LINES TERMINATED BY '${rowSep}' + |STORED AS TEXTFILE + |LOCATION '${tablePath}'""".stripMargin + sqlContext.sql(sql) + } catch { + case e: Throwable => throw e } - val sql = s"""CREATE EXTERNAL TABLE IF NOT EXISTS `${tableName}` - |(${colsSql}) PARTITIONED BY (${partitionsSql}) - |ROW FORMAT DELIMITED - |FIELDS TERMINATED BY '${fieldSep}' - |LINES TERMINATED BY '${rowSep}' - |STORED AS TEXTFILE - |LOCATION '${tablePath}'""".stripMargin - sqlContext.sql(sql) } def available(): Boolean = { true } - private def encode(data: Map[String, Any]): Schema = { - ; - } - - private def decode(data: Schema): Map[String, Any] = { - ; + private def encode(data: Map[String, Any], ms: Long): Option[List[Any]] = { + try { + Some(schema.map { field => + val (name, _) = field + name match { + case TimeStampColumn => ms + case PayloadColumn => JsonUtil.toJson(data) + case _ => null + } + }) + } catch { + case _ => None + } } - private def getPartition(ms: Long): List[(String, Any)] = { - partition.map { p => - val (name, _, unit) = p - val t = TimeUtil.timeToUnit(ms, unit) - (name, t) + private def decode(data: List[Any], updateTimeStamp: Boolean): Option[Map[String, Any]] = { + val dataMap = schemaName.zip(data).toMap + dataMap.get(PayloadColumn) match { + case Some(v: String) => { + try { + val map = JsonUtil.toAnyMap(v) + val resMap = if (updateTimeStamp) { + dataMap.get(TimeStampColumn) match { + case Some(t) => map + (TimeStampColumn -> t) + case _ => map + } + } else map + Some(resMap) + } catch { + case _ => None + } + } + case _ => None } } - private def genPartitionHdfsPath(partitions: List[(String, Any)]): String = { - partitions.map(prtn => s"${prtn._1}=${prtn._2}").mkString("/") - } def saveData(rdd: RDD[Map[String, Any]], ms: Long): Unit = { val newCacheLocked = newCacheLock.lock(-1, TimeUnit.SECONDS) if (newCacheLocked) { try { val ptns = getPartition(ms) - val ptnsSql = ptns.map(ptn => (s"`${ptn._1}`=${ptn._2}")).mkString(", ") val ptnsPath = genPartitionHdfsPath(ptns) - val filePath = s"${tablePath}/${ptnsPath}/${ms}" + val dirPath = s"${tablePath}/${ptnsPath}" + val fileName = s"${ms}" + val filePath = HdfsUtil.getHdfsFilePath(dirPath, fileName) // encode data - val dataRdd = rdd.map(encode(_)) + val dataRdd: RDD[List[Any]] = rdd.flatMap(encode(_, ms)) + + // save data + val recordRdd: RDD[String] = dataRdd.map { dt => + dt.map(_.toString).mkString(fieldSep) + } - // save data to hdfs - // fixme: waiting... + val dumped = if (!recordRdd.isEmpty) { + HdfsFileDumpUtil.dump(filePath, recordRdd, rowSepLiteral) + } else false - // add partition info + // add partition + if (dumped) { + val sql = addPartitionSql(concreteTableName, ptns) + sqlContext.sql(sql) + } // submit ms submitCacheTime(ms) submitReadyTime(ms) + } catch { + case e: Throwable => error(s"save data error: ${e.getMessage}") } finally { newCacheLock.unlock() } } - // if (newDataFrame == null) { - // newDataFrame = df - // newDataFrame.persist(StorageLevel.fromString(cacheLevel)) - // } else { - // if (!df.rdd.isEmpty) { - // newDataFrame.unpersist() - // newDataFrame = newDataFrame.unionAll(df) - // newDataFrame.persist(StorageLevel.fromString(cacheLevel)) - // } - // } - // // submit ms - // submitCacheTime(ms) - // submitReadyTime(ms) } def readData(): Try[RDD[Map[String, Any]]] = Try { - // if (initialed) { - // val timeRange = TimeInfoCache.getTimeRange - // println(s"timeRange: ${timeRange}") - // submitLastProcTime(timeRange._2) - // - // val reviseTimeRange = (timeRange._1 + deltaTimeRange._1, timeRange._2 + deltaTimeRange._2) - // println(s"reviseTimeRange: ${reviseTimeRange}") - // dataFrame.filter(s"${timeStampColumn} BETWEEN ${reviseTimeRange._1} AND ${reviseTimeRange._2}") - // } else { - // throw new Exception("data not cached") - // } - val timeRange = TimeInfoCache.getTimeRange println(s"timeRange: ${timeRange}") submitLastProcTime(timeRange._2) + val reviseTimeRange = (timeRange._1 + deltaTimeRange._1, timeRange._2 + deltaTimeRange._2) println(s"reviseTimeRange: ${reviseTimeRange}") - // move new data frame to temp data frame - val newCacheLocked = newCacheLock.lock(-1, TimeUnit.SECONDS) - val newTempDataFrame = if (newCacheLocked) { - try { - val tmp = newDataFrame.filter(s"${timeStampColumn} BETWEEN ${reviseTimeRange._1} AND ${reviseTimeRange._2}") - newDataFrame.unpersist() - newDataFrame = newDataFrame.filter(s"${timeStampColumn} > ${reviseTimeRange._2}") - tmp - } finally { - newCacheLock.unlock() - } - } else null - - // add temp data frame to old data frame - val oldCacheLocked = oldCacheLock.lock(-1, TimeUnit.SECONDS) - val oldTempDataFrame = if (oldCacheLocked) { - try { - if (oldDataFrame != null) { - oldDataFrame.filter(s"${timeStampColumn} BETWEEN ${reviseTimeRange._1} AND ${reviseTimeRange._2}") - } else null - } finally { - oldCacheLock.unlock() - } - } else { - throw new Exception("old cache lock unavailable") - } + // read directly through partition info + val partitionRange = getPartitionRange(reviseTimeRange._1, reviseTimeRange._2) + val sql = selectSql(concreteTableName, partitionRange) + val df = sqlContext.sql(sql) - if (oldTempDataFrame == null && newTempDataFrame == null) { - throw new Exception("data not cached") - } else { - val finalDataFrame = if (newTempDataFrame == null) { - oldTempDataFrame - } else if (oldTempDataFrame == null) { - newTempDataFrame - } else { - oldTempDataFrame.unionAll(newTempDataFrame) + // decode data + df.flatMap { row => + val dt = schemaName.map { sn => + row.getAs[Any](sn) } - finalDataFrame + decode(dt, true) } } @@ -245,47 +256,96 @@ case class HiveCacheDataConnector(sqlContext: SQLContext, dataCacheParam: DataCa val reviseTimeRange = (timeRange._1 + deltaTimeRange._1, timeRange._2 + deltaTimeRange._2) println(s"clean reviseTimeRange: ${reviseTimeRange}") - oldDataFrame.unpersist() - oldDataFrame = oldDataFrame.filter(s"${timeStampColumn} >= ${reviseTimeRange._1}") - oldDataFrame.persist(StorageLevel.fromString(cacheLevel)) + // drop partition + val lowerBound = getPartition(reviseTimeRange._1) + val sql = dropPartitionSql(concreteTableName, lowerBound) + sqlContext.sql(sql) + + // fixme: remove data +// readCleanTime match { +// case Some(ct) => { +// ; +// } +// case _ => { +// ; +// } +// } + } catch { + case e: Throwable => error(s"clean old data error: ${e.getMessage}") } finally { oldCacheLock.unlock() } } - - // if (initialed) { - // val timeRange = TimeInfoCache.getTimeRange - // val reviseTimeRange = (timeRange._1 + deltaTimeRange._1, timeRange._2 + deltaTimeRange._2) - // println(s"clean reviseTimeRange: ${reviseTimeRange}") - // - // dataFrame.show(10) - // - // dataFrame.unpersist() - // dataFrame = dataFrame.filter(s"${timeStampColumn} >= ${reviseTimeRange._1}") - // dataFrame.persist(StorageLevel.fromString(cacheLevel)) - // - // dataFrame.show(10) - // } } - override def updateOldData(oldRdd: RDD[Map[String, Any]]): Unit = { - val oldCacheLocked = oldCacheLock.lock(-1, TimeUnit.SECONDS) - if (oldCacheLocked) { - try { - if (oldDataFrame == null) { - oldDataFrame = oldDf - oldDataFrame.persist(StorageLevel.fromString(cacheLevel)) - } else { - if (!oldDf.rdd.isEmpty) { - oldDataFrame.unpersist() - oldDataFrame = oldDf - oldDataFrame.persist(StorageLevel.fromString(cacheLevel)) - } + override def updateOldData(t: Long, oldData: Iterable[Map[String, Any]]): Unit = { + // parallel process different time groups, lock is unnecessary + val ptns = getPartition(t) + val ptnsPath = genPartitionHdfsPath(ptns) + val dirPath = s"${tablePath}/${ptnsPath}" + val fileName = s"${t}" + val filePath = HdfsUtil.getHdfsFilePath(dirPath, fileName) + + try { + // remove out time old data + HdfsFileDumpUtil.remove(dirPath, fileName, true) + + // save updated old data + if (oldData.size > 0) { + val recordDatas = oldData.flatMap { dt => + encode(dt, t) } - } finally { - oldCacheLock.unlock() + val records: Iterable[String] = recordDatas.map { dt => + dt.map(_.toString).mkString(fieldSep) + } + val dumped = HdfsFileDumpUtil.dump(filePath, records, rowSepLiteral) } + } catch { + case e: Throwable => error(s"update old data error: ${e.getMessage}") } } + private def getPartition(ms: Long): List[(String, Any)] = { + partition.map { p => + val (name, _, unit) = p + val t = TimeUtil.timeToUnit(ms, unit) + (name, t) + } + } + private def getPartitionRange(ms1: Long, ms2: Long): List[(String, (Any, Any))] = { + partition.map { p => + val (name, _, unit) = p + val t1 = TimeUtil.timeToUnit(ms1, unit) + val t2 = TimeUtil.timeToUnit(ms2, unit) + (name, (t1, t2)) + } + } + + private def genPartitionHdfsPath(partition: List[(String, Any)]): String = { + partition.map(prtn => s"${prtn._1}=${prtn._2}").mkString("/") + } + private def addPartitionSql(tbn: String, partition: List[(String, Any)]): String = { + val partitionSql = partition.map(ptn => (s"`${ptn._1}` = ${ptn._2}")).mkString(", ") + val sql = s"""ALTER TABLE ${tbn} ADD IF NOT EXISTS PARTITION (${partitionSql})""" + sql + } + private def selectSql(tbn: String, partitionRange: List[(String, (Any, Any))]): String = { + val clause = partitionRange.map { pr => + val (name, (r1, r2)) = pr + s"""`${name}` BETWEEN '${r1}' and '${r2}'""" + }.mkString(" AND ") + val whereClause = if (clause.nonEmpty) s"WHERE ${clause}" else "" + val sql = s"""SELECT * FROM ${tbn} ${whereClause}""" + sql + } + private def dropPartitionSql(tbn: String, partition: List[(String, Any)]): String = { + val partitionSql = partition.map(ptn => (s"PARTITION ( `${ptn._1}` < ${ptn._2} ) ")).mkString(", ") + val sql = s"""ALTER TABLE ${tbn} DROP ${partitionSql}""" + sql + } + + private def tableExistsSql(): String = { + s"tableName LIKE '${tableName}'" + } + } diff --git a/measure/src/main/scala/org/apache/griffin/measure/connector/KafkaDataConnector.scala b/measure/src/main/scala/org/apache/griffin/measure/connector/KafkaDataConnector.scala index a624ccc8e..532f84e19 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/connector/KafkaDataConnector.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/connector/KafkaDataConnector.scala @@ -189,9 +189,15 @@ case class KafkaDataConnector(sqlContext: SQLContext, @transient ssc: StreamingC cacheDataConnector.cleanOldData } - override def updateOldData(oldRdd: RDD[Map[String, Any]]): Unit = { + override def updateOldData(t: Long, oldData: Iterable[Map[String, Any]]): Unit = { if (dataConnectorParam.getMatchOnce) { - cacheDataConnector.updateOldData(oldRdd) + cacheDataConnector.updateOldData(t, oldData) + } + } + + override def updateAllOldData(oldRdd: RDD[Map[String, Any]]): Unit = { + if (dataConnectorParam.getMatchOnce) { + cacheDataConnector.updateAllOldData(oldRdd) } } diff --git a/measure/src/main/scala/org/apache/griffin/measure/persist/HdfsPersist.scala b/measure/src/main/scala/org/apache/griffin/measure/persist/HdfsPersist.scala index ff6bb67f1..cbaa4c2bc 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/persist/HdfsPersist.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/persist/HdfsPersist.scala @@ -69,12 +69,8 @@ case class HdfsPersist(config: Map[String, Any], metricName: String, timeStamp: s"--- ${dt} ---\n" } - protected def getFilePath(parentPath: String, fileName: String): String = { - if (parentPath.endsWith(separator)) parentPath + fileName else parentPath + separator + fileName - } - protected def filePath(file: String): String = { - getFilePath(path, s"${metricName}/${timeStamp}/${file}") + HdfsUtil.getHdfsFilePath(path, s"${metricName}/${timeStamp}/${file}") } protected def withSuffix(path: String, suffix: String): String = { diff --git a/measure/src/main/scala/org/apache/griffin/measure/utils/HdfsFileDumpUtil.scala b/measure/src/main/scala/org/apache/griffin/measure/utils/HdfsFileDumpUtil.scala index ede6e39ab..fc3860f8a 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/utils/HdfsFileDumpUtil.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/utils/HdfsFileDumpUtil.scala @@ -27,11 +27,18 @@ object HdfsFileDumpUtil { private def suffix(i: Long): String = { if (i == 0) "" else s".${i}" } + private def samePattern(fileName: String, patternFileName: String): Boolean = { + fileName.startsWith(patternFileName) + } def splitRdd[T](rdd: RDD[T])(implicit m: Manifest[T]): RDD[(Long, Iterable[T])] = { val indexRdd = rdd.zipWithIndex indexRdd.map(p => ((p._2 / sepCount), p._1)).groupByKey() } + def splitIterable[T](datas: Iterable[T])(implicit m: Manifest[T]): Iterator[(Int, Iterable[T])] = { + val groupedData = datas.grouped(sepCount).zipWithIndex + groupedData.map(v => (v._2, v._1)) + } private def directDump(path: String, list: Iterable[String], lineSep: String): Unit = { // collect and save @@ -49,5 +56,28 @@ object HdfsFileDumpUtil { true }, _ && _) } + def dump(path: String, records: Iterable[String], lineSep: String): Boolean = { + val groupedRecords = splitIterable(records) + groupedRecords.aggregate(true)({ (res, pair) => + val (idx, list) = pair + val filePath = path + suffix(idx) + directDump(filePath, list, lineSep) + true + }, _ && _) + } + + def remove(path: String, filename: String, withSuffix: Boolean): Unit = { + if (withSuffix) { + val files = HdfsUtil.listPathFiles(path) + val patternFiles = files.filter(samePattern(_, filename)) + patternFiles.foreach { f => + val rmPath = HdfsUtil.getHdfsFilePath(path, f) + HdfsUtil.deleteHdfsPath(rmPath) + } + } else { + val rmPath = HdfsUtil.getHdfsFilePath(path, filename) + HdfsUtil.deleteHdfsPath(rmPath) + } + } } diff --git a/measure/src/main/scala/org/apache/griffin/measure/utils/HdfsUtil.scala b/measure/src/main/scala/org/apache/griffin/measure/utils/HdfsUtil.scala index 89556846f..8b8d15f85 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/utils/HdfsUtil.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/utils/HdfsUtil.scala @@ -27,6 +27,7 @@ object HdfsUtil { private val conf = new Configuration() conf.set("dfs.support.append", "true") + conf.set("fs.defaultFS", "hdfs://localhost") // fixme private val dfs = FileSystem.get(conf) @@ -77,4 +78,16 @@ object HdfsUtil { val path = new Path(dirPath) if (dfs.exists(path)) dfs.delete(path, true) } + + def listPathFiles(dirPath: String): Iterable[String] = { + val path = new Path(dirPath) + try { + val fileStatusArray = dfs.listStatus(path) + fileStatusArray.flatMap { fileStatus => + if (fileStatus.isFile) { + Some(fileStatus.getPath.getName) + } else None + } + } + } } diff --git a/measure/src/main/scala/org/apache/griffin/measure/utils/JsonUtil.scala b/measure/src/main/scala/org/apache/griffin/measure/utils/JsonUtil.scala index 76d5d6d89..14183759d 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/utils/JsonUtil.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/utils/JsonUtil.scala @@ -48,7 +48,7 @@ object JsonUtil { mapper.readValue[T](is, classTag[T].runtimeClass.asInstanceOf[Class[T]]) } - def toAnyMap(json: String) = { + def toAnyMap(json: String): Map[String, Any] = { mapper.readValue(json, classOf[Map[String, Any]]) } } diff --git a/measure/src/test/resources/config-streaming.json b/measure/src/test/resources/config-streaming.json index 93c410feb..8a26887af 100644 --- a/measure/src/test/resources/config-streaming.json +++ b/measure/src/test/resources/config-streaming.json @@ -19,6 +19,7 @@ "cache": { "type": "hive", "config": { + "database": "griffin_cache", "table.name": "src", "parent.path": "hdfs://localhost/griffin/streaming/dump", "info.path": "src", @@ -47,6 +48,7 @@ "cache": { "type": "hive", "config": { + "database": "griffin_cache", "table.name": "tgt", "parent.path": "hdfs://localhost/griffin/streaming/dump", "info.path": "tgt", diff --git a/measure/src/test/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgoTest.scala b/measure/src/test/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgoTest.scala index 090c6f58a..2aabf4a07 100644 --- a/measure/src/test/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgoTest.scala +++ b/measure/src/test/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgoTest.scala @@ -35,7 +35,7 @@ import org.apache.griffin.measure.persist.{Persist, PersistFactory, PersistType} import org.apache.griffin.measure.result._ import org.apache.griffin.measure.rule.expr._ import org.apache.griffin.measure.rule.{ExprValueUtil, RuleAnalyzer, RuleFactory} -import org.apache.griffin.measure.utils.TimeUtil +import org.apache.griffin.measure.utils.{HdfsUtil, TimeUtil} import org.apache.spark.rdd.RDD import org.apache.spark.sql.SQLContext import org.apache.spark.sql.hive.HiveContext @@ -52,7 +52,7 @@ import scala.util.{Failure, Success, Try} class StreamingAccuracyAlgoTest extends FunSuite with Matchers with BeforeAndAfter with Loggable { val envFile = "src/test/resources/env-streaming.json" - val confFile = "src/test/resources/config-streaming1.json" + val confFile = "src/test/resources/config-streaming.json" val envFsType = "local" val userFsType = "local" @@ -100,9 +100,12 @@ class StreamingAccuracyAlgoTest extends FunSuite with Matchers with BeforeAndAft // sqlContext = new SQLContext(sc) sqlContext = new HiveContext(sc) - val a = sqlContext.sql("select * from test limit 10") + val a = sqlContext.sql("select * from s1 limit 10") // val a = sqlContext.sql("show tables") a.show(10) + + val b = HdfsUtil.existPath("/griffin/streaming") + println(b) } test("algorithm") { @@ -239,42 +242,60 @@ class StreamingAccuracyAlgoTest extends FunSuite with Matchers with BeforeAndAft val updatedCacheResultOpt = cacheResultProcesser.genUpdateCacheResult(t, updateTime, res) - updatedCacheResultOpt match { - case Some(updatedCacheResult) => { - val persist: Persist = persistFactory.getPersists(t) - - // persist result - persist.result(updateTime, accuResult) - - // persist missing data - val missStrings = missData.map { row => - val (key, (value, info)) = row - s"${value} [${info.getOrElse(MismatchInfo.key, "unknown")}]" - } - persist.records(missStrings, PersistType.MISS) - } - case _ => { - // do nothing - } + updatedCacheResultOpt.flatMap { updatedCacheResult => + Some((updatedCacheResult, (t, missData))) } + } + + updateResults.cache - updatedCacheResultOpt - }.collect() + val updateResultsPart = updateResults.map(_._1) + val updateDataPart = updateResults.map(_._2) - // update results cache together - updateResults.foreach { updateResult => + val updateResultsArray = updateResultsPart.collect() + + // update results cache (in driver) + // collect action is traversable once action, it will make rdd updateResults empty + updateResultsArray.foreach { updateResult => + println(s"update result: ${updateResult}") cacheResultProcesser.update(updateResult) + // persist result + val persist: Persist = persistFactory.getPersists(updateResult.timeGroup) + persist.result(updateTime, updateResult.result) } - // dump missing rdd - val dumpRdd: RDD[Map[String, Any]] = missingRdd.map { row => - val (key, (value, info)) = row - value ++ info + // record missing data and update old data (in executor) + updateDataPart.foreach { grp => + val (t, datas) = grp + val persist: Persist = persistFactory.getPersists(t) + // persist missing data + val missStrings = datas.map { row => + val (_, (value, info)) = row + s"${value} [${info.getOrElse(MismatchInfo.key, "unknown")}]" + } + persist.records(missStrings, PersistType.MISS) + // data connector update old data + val dumpDatas = datas.map { r => + val (_, (v, i)) = r + v ++ i + } + + println(t) + dumpDatas.foreach(println) + + sourceDataConnector.updateOldData(t, dumpDatas) + targetDataConnector.updateOldData(t, dumpDatas) // not correct } - sourceDataConnector.updateOldData(dumpRdd) - targetDataConnector.updateOldData(dumpRdd) // not correct - // fixme: 2. implement the hive cahe data connector + updateResults.unpersist() + + // dump missing rdd (this part not need for future version, only for current df cache data version) + val dumpRdd: RDD[Map[String, Any]] = missingRdd.map { r => + val (_, (v, i)) = r + v ++ i + } + sourceDataConnector.updateAllOldData(dumpRdd) + targetDataConnector.updateAllOldData(dumpRdd) // not correct // fixme: 3. refactor data connector module diff --git a/measure/src/test/scala/org/apache/griffin/measure/connector/ConnectorTest.scala b/measure/src/test/scala/org/apache/griffin/measure/connector/ConnectorTest.scala index 388f5d35b..48771ab75 100644 --- a/measure/src/test/scala/org/apache/griffin/measure/connector/ConnectorTest.scala +++ b/measure/src/test/scala/org/apache/griffin/measure/connector/ConnectorTest.scala @@ -30,7 +30,7 @@ import org.apache.griffin.measure.config.reader.ParamRawStringReader import org.apache.griffin.measure.result.{DataInfo, TimeStampInfo} import org.apache.griffin.measure.rule.expr.{Expr, StatementExpr} import org.apache.griffin.measure.rule._ -import org.apache.griffin.measure.utils.TimeUtil +import org.apache.griffin.measure.utils.{HdfsFileDumpUtil, HdfsUtil, TimeUtil} import org.apache.griffin.measure.rule.{DataTypeCalculationUtil, ExprValueUtil, RuleExprs} import org.apache.spark.rdd.RDD import org.apache.spark.sql.types.{DataType, StructField, StructType} @@ -309,8 +309,5 @@ class ConnectorTest extends FunSuite with Matchers with BeforeAndAfter { process.shutdown() } - - - } From 581a7a7d8350bcb951a4bd45d7547e113d0ce562 Mon Sep 17 00:00:00 2001 From: Lionel Liu Date: Fri, 21 Jul 2017 16:17:26 +0800 Subject: [PATCH 027/111] v1 --- .../algo/batch/BatchAccuracyAlgo.scala | 9 +- .../measure/algo/batch/BatchProfileAlgo.scala | 5 +- .../streaming/StreamingAccuracyAlgo.scala | 9 +- .../measure/connector/DataConnector.scala | 7 +- .../connector/DataConnectorFactory.scala | 57 +-- .../connector/KafkaDataConnector.scala | 246 ---------- .../connector/cache/CacheDataConnector.scala | 33 ++ .../DataCacheable.scala} | 15 +- .../connector/cache/DataUpdatable.scala | 30 ++ .../{ => cache}/DfCacheDataConnector.scala | 9 +- .../{ => cache}/HiveCacheDataConnector.scala | 5 +- .../AvroDirectDataConnector.scala} | 22 +- .../DirectDataConnector.scala} | 6 +- .../HiveDirectDataConnector.scala} | 13 +- .../KafkaCacheDirectDataConnector.scala | 125 +++++ .../StreamingCacheDirectDataConnector.scala | 60 +++ .../KafkaStreamingDataConnector.scala | 15 +- .../StreamingDataConnector.scala | 6 +- .../griffin/measure/utils/HdfsUtil.scala | 2 +- .../algo/batch/BatchAccuracyAlgoTest.scala | 11 +- .../algo/batch/BatchProfileAlgoTest.scala | 7 +- .../algo/batch/DataFrameSaveTest.scala | 7 +- .../streaming/StreamingAccuracyAlgoTest.scala | 11 +- .../measure/connector/ConnectorTest.scala | 432 +++++++++--------- 24 files changed, 567 insertions(+), 575 deletions(-) delete mode 100644 measure/src/main/scala/org/apache/griffin/measure/connector/KafkaDataConnector.scala create mode 100644 measure/src/main/scala/org/apache/griffin/measure/connector/cache/CacheDataConnector.scala rename measure/src/main/scala/org/apache/griffin/measure/connector/{CacheDataConnector.scala => cache/DataCacheable.scala} (87%) create mode 100644 measure/src/main/scala/org/apache/griffin/measure/connector/cache/DataUpdatable.scala rename measure/src/main/scala/org/apache/griffin/measure/connector/{ => cache}/DfCacheDataConnector.scala (98%) rename measure/src/main/scala/org/apache/griffin/measure/connector/{ => cache}/HiveCacheDataConnector.scala (98%) rename measure/src/main/scala/org/apache/griffin/measure/connector/{AvroBatchDataConnector.scala => direct/AvroDirectDataConnector.scala} (89%) rename measure/src/main/scala/org/apache/griffin/measure/connector/{BatchDataConnector.scala => direct/DirectDataConnector.scala} (79%) rename measure/src/main/scala/org/apache/griffin/measure/connector/{HiveBatchDataConnector.scala => direct/HiveDirectDataConnector.scala} (92%) create mode 100644 measure/src/main/scala/org/apache/griffin/measure/connector/direct/KafkaCacheDirectDataConnector.scala create mode 100644 measure/src/main/scala/org/apache/griffin/measure/connector/direct/StreamingCacheDirectDataConnector.scala rename measure/src/main/scala/org/apache/griffin/measure/connector/{ => streaming}/KafkaStreamingDataConnector.scala (75%) rename measure/src/main/scala/org/apache/griffin/measure/connector/{ => streaming}/StreamingDataConnector.scala (84%) diff --git a/measure/src/main/scala/org/apache/griffin/measure/algo/batch/BatchAccuracyAlgo.scala b/measure/src/main/scala/org/apache/griffin/measure/algo/batch/BatchAccuracyAlgo.scala index 28e89ef6f..9240444a2 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/algo/batch/BatchAccuracyAlgo.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/algo/batch/BatchAccuracyAlgo.scala @@ -24,6 +24,7 @@ import org.apache.griffin.measure.algo.AccuracyAlgo import org.apache.griffin.measure.algo.core.AccuracyCore import org.apache.griffin.measure.config.params.AllParam import org.apache.griffin.measure.connector._ +import org.apache.griffin.measure.connector.direct.DirectDataConnector import org.apache.griffin.measure.persist._ import org.apache.griffin.measure.result._ import org.apache.griffin.measure.rule._ @@ -76,8 +77,8 @@ case class BatchAccuracyAlgo(allParam: AllParam) extends AccuracyAlgo { } // data connector - val sourceDataConnector: BatchDataConnector = - DataConnectorFactory.getBatchDataConnector(sqlContext, userParam.sourceParam, + val sourceDataConnector: DirectDataConnector = + DataConnectorFactory.getDirectDataConnector(sqlContext, null, userParam.sourceParam, ruleAnalyzer.sourceRuleExprs, finalConstMap ) match { case Success(cntr) => { @@ -86,8 +87,8 @@ case class BatchAccuracyAlgo(allParam: AllParam) extends AccuracyAlgo { } case Failure(ex) => throw ex } - val targetDataConnector: BatchDataConnector = - DataConnectorFactory.getBatchDataConnector(sqlContext, userParam.targetParam, + val targetDataConnector: DirectDataConnector = + DataConnectorFactory.getDirectDataConnector(sqlContext, null, userParam.targetParam, ruleAnalyzer.targetRuleExprs, finalConstMap ) match { case Success(cntr) => { diff --git a/measure/src/main/scala/org/apache/griffin/measure/algo/batch/BatchProfileAlgo.scala b/measure/src/main/scala/org/apache/griffin/measure/algo/batch/BatchProfileAlgo.scala index 1672e61e7..bc15a508d 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/algo/batch/BatchProfileAlgo.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/algo/batch/BatchProfileAlgo.scala @@ -24,6 +24,7 @@ import org.apache.griffin.measure.algo.ProfileAlgo import org.apache.griffin.measure.algo.core.ProfileCore import org.apache.griffin.measure.config.params._ import org.apache.griffin.measure.connector._ +import org.apache.griffin.measure.connector.direct.DirectDataConnector import org.apache.griffin.measure.persist.{Persist, PersistFactory, PersistType} import org.apache.griffin.measure.result._ import org.apache.griffin.measure.rule.expr._ @@ -76,8 +77,8 @@ case class BatchProfileAlgo(allParam: AllParam) extends ProfileAlgo { } // data connector - val sourceDataConnector: BatchDataConnector = - DataConnectorFactory.getBatchDataConnector(sqlContext, userParam.sourceParam, + val sourceDataConnector: DirectDataConnector = + DataConnectorFactory.getDirectDataConnector(sqlContext, null, userParam.sourceParam, ruleAnalyzer.sourceRuleExprs, finalConstMap ) match { case Success(cntr) => { diff --git a/measure/src/main/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgo.scala b/measure/src/main/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgo.scala index 5d81ba55f..d974ebcbd 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgo.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgo.scala @@ -26,6 +26,7 @@ import org.apache.griffin.measure.algo.core.AccuracyCore import org.apache.griffin.measure.cache.info.InfoCacheInstance import org.apache.griffin.measure.config.params.AllParam import org.apache.griffin.measure.connector._ +import org.apache.griffin.measure.connector.direct.DirectDataConnector import org.apache.griffin.measure.persist.{Persist, PersistFactory} import org.apache.griffin.measure.result.{AccuracyResult, TimeStampInfo} import org.apache.griffin.measure.rule.{ExprValueUtil, RuleAnalyzer, RuleFactory} @@ -87,8 +88,8 @@ case class StreamingAccuracyAlgo(allParam: AllParam) extends AccuracyAlgo { } // data connector - val sourceDataConnector: BatchDataConnector = - DataConnectorFactory.getDataConnector(sqlContext, ssc, userParam.sourceParam, + val sourceDataConnector: DirectDataConnector = + DataConnectorFactory.getDirectDataConnector(sqlContext, ssc, userParam.sourceParam, ruleAnalyzer.sourceRuleExprs, finalConstMap ) match { case Success(cntr) => { @@ -97,8 +98,8 @@ case class StreamingAccuracyAlgo(allParam: AllParam) extends AccuracyAlgo { } case Failure(ex) => throw ex } - val targetDataConnector: BatchDataConnector = - DataConnectorFactory.getDataConnector(sqlContext, ssc, userParam.targetParam, + val targetDataConnector: DirectDataConnector = + DataConnectorFactory.getDirectDataConnector(sqlContext, ssc, userParam.targetParam, ruleAnalyzer.targetRuleExprs, finalConstMap ) match { case Success(cntr) => { diff --git a/measure/src/main/scala/org/apache/griffin/measure/connector/DataConnector.scala b/measure/src/main/scala/org/apache/griffin/measure/connector/DataConnector.scala index 527676be7..1fb1868fd 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/connector/DataConnector.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/connector/DataConnector.scala @@ -27,11 +27,6 @@ trait DataConnector extends Loggable with Serializable { def available(): Boolean - def init(): Unit = {} - - def cleanOldData(): Unit = {} - - def updateOldData(t: Long, oldData: Iterable[Map[String, Any]]): Unit = {} - def updateAllOldData(oldRdd: RDD[Map[String, Any]]): Unit = {} + def init(): Unit } diff --git a/measure/src/main/scala/org/apache/griffin/measure/connector/DataConnectorFactory.scala b/measure/src/main/scala/org/apache/griffin/measure/connector/DataConnectorFactory.scala index 41b90cd57..72456d8b4 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/connector/DataConnectorFactory.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/connector/DataConnectorFactory.scala @@ -20,6 +20,9 @@ package org.apache.griffin.measure.connector import kafka.serializer.StringDecoder import org.apache.griffin.measure.config.params.user._ +import org.apache.griffin.measure.connector.cache._ +import org.apache.griffin.measure.connector.direct._ +import org.apache.griffin.measure.connector.streaming._ import org.apache.griffin.measure.rule.RuleExprs import org.apache.spark.sql.SQLContext import org.apache.spark.streaming.StreamingContext @@ -38,45 +41,32 @@ object DataConnectorFactory { val DfRegex = """^(?i)df|dataframe$""".r - def getDataConnector(sqlContext: SQLContext, - ssc: StreamingContext, - dataConnectorParam: DataConnectorParam, - ruleExprs: RuleExprs, - globalFinalCacheMap: Map[String, Any] - ): Try[BatchDataConnector] = { + def getDirectDataConnector(sqlContext: SQLContext, + ssc: StreamingContext, + dataConnectorParam: DataConnectorParam, + ruleExprs: RuleExprs, + globalFinalCacheMap: Map[String, Any] + ): Try[DirectDataConnector] = { val conType = dataConnectorParam.conType val version = dataConnectorParam.version val config = dataConnectorParam.config Try { conType match { - case HiveRegex() => HiveBatchDataConnector(sqlContext, config, ruleExprs, globalFinalCacheMap) - case AvroRegex() => AvroBatchDataConnector(sqlContext, config, ruleExprs, globalFinalCacheMap) - case KafkaRegex() => KafkaDataConnector(sqlContext, ssc, dataConnectorParam, ruleExprs, globalFinalCacheMap) + case HiveRegex() => HiveDirectDataConnector(sqlContext, config, ruleExprs, globalFinalCacheMap) + case AvroRegex() => AvroDirectDataConnector(sqlContext, config, ruleExprs, globalFinalCacheMap) + case KafkaRegex() => { + val ksdcTry = getStreamingDataConnector(ssc, dataConnectorParam) + val cdcTry = getCacheDataConnector(sqlContext, dataConnectorParam.cache) + KafkaCacheDirectDataConnector(ksdcTry, cdcTry, dataConnectorParam, ruleExprs, globalFinalCacheMap) + } case _ => throw new Exception("connector creation error!") } } } - def getBatchDataConnector(sqlContext: SQLContext, - dataConnectorParam: DataConnectorParam, - ruleExprs: RuleExprs, - globalFinalCacheMap: Map[String, Any] - ): Try[BatchDataConnector] = { - val conType = dataConnectorParam.conType - val version = dataConnectorParam.version - val config = dataConnectorParam.config - Try { - conType match { - case HiveRegex() => HiveBatchDataConnector(sqlContext, config, ruleExprs, globalFinalCacheMap) - case AvroRegex() => AvroBatchDataConnector(sqlContext, config, ruleExprs, globalFinalCacheMap) - case _ => throw new Exception("batch connector creation error!") - } - } - } - - def getStreamingDataConnector(ssc: StreamingContext, - dataConnectorParam: DataConnectorParam - ): Try[StreamingDataConnector] = { + private def getStreamingDataConnector(ssc: StreamingContext, + dataConnectorParam: DataConnectorParam + ): Try[StreamingDataConnector] = { val conType = dataConnectorParam.conType val version = dataConnectorParam.version val config = dataConnectorParam.config @@ -90,9 +80,9 @@ object DataConnectorFactory { } } - def getCacheDataConnector(sqlContext: SQLContext, - dataCacheParam: DataCacheParam - ): Try[CacheDataConnector] = { + private def getCacheDataConnector(sqlContext: SQLContext, + dataCacheParam: DataCacheParam + ): Try[CacheDataConnector] = { if (dataCacheParam == null) { throw new Exception("invalid data cache param!") } @@ -106,7 +96,7 @@ object DataConnectorFactory { } } - protected def genKafkaDataConnector(ssc: StreamingContext, config: Map[String, Any]) = { + private def genKafkaDataConnector(ssc: StreamingContext, config: Map[String, Any]) = { val KeyType = "key.type" val ValueType = "value.type" val keyType = config.getOrElse(KeyType, "java.lang.String").toString @@ -120,6 +110,7 @@ object DataConnectorFactory { // val topics = config.getOrElse(Topics, "").toString (getClassTag(keyType), getClassTag(valueType)) match { case (ClassTag(k: Class[String]), ClassTag(v: Class[String])) => { + if (ssc == null) throw new Exception("streaming context is null! ") new KafkaStreamingDataConnector(ssc, config) { type K = String type KD = StringDecoder diff --git a/measure/src/main/scala/org/apache/griffin/measure/connector/KafkaDataConnector.scala b/measure/src/main/scala/org/apache/griffin/measure/connector/KafkaDataConnector.scala deleted file mode 100644 index 532f84e19..000000000 --- a/measure/src/main/scala/org/apache/griffin/measure/connector/KafkaDataConnector.scala +++ /dev/null @@ -1,246 +0,0 @@ -/* -Licensed to the Apache Software Foundation (ASF) under one -or more contributor license agreements. See the NOTICE file -distributed with this work for additional information -regarding copyright ownership. The ASF licenses this file -to you under the Apache License, Version 2.0 (the -"License"); you may not use this file except in compliance -with the License. You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, -software distributed under the License is distributed on an -"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -KIND, either express or implied. See the License for the -specific language governing permissions and limitations -under the License. -*/ -package org.apache.griffin.measure.connector - -import java.util.Date - -import org.apache.griffin.measure.config.params.user.{DataConnectorParam, EvaluateRuleParam} -import org.apache.griffin.measure.result._ -import org.apache.griffin.measure.rule.expr.StatementExpr -import org.apache.griffin.measure.rule._ -import org.apache.griffin.measure.utils.TimeUtil -import org.apache.spark.rdd.RDD -import org.apache.spark.sql.types.{DataType, StructField, StructType} -import org.apache.spark.streaming.StreamingContext -import org.apache.spark.sql.{DataFrame, Row, SQLContext} - -import scala.util.{Failure, Success, Try} - -case class KafkaDataConnector(sqlContext: SQLContext, @transient ssc: StreamingContext, dataConnectorParam: DataConnectorParam, - ruleExprs: RuleExprs, constFinalExprValueMap: Map[String, Any] - ) extends BatchDataConnector { - - @transient val kafkaStreamingDataConnector = DataConnectorFactory.getStreamingDataConnector(ssc, dataConnectorParam) match { - case Success(cntr) => cntr - case Failure(ex) => throw ex - } - - val dataCacheParam = dataConnectorParam.cache - val cacheDataConnector = DataConnectorFactory.getCacheDataConnector(sqlContext, dataCacheParam) match { - case Success(cntr) => cntr - case Failure(ex) => throw ex - } - -// val DumpDatabase = "dump.database" -// val DumpTableName = "dump.table.name" -// val TempTableName = "temp.table.name" -// val TableNameRegex = """^[a-zA-Z\d][\w#@]{0,127}$""".r -// -// val dumpDatabase = dataConnectorParam.config.getOrElse(DumpDatabase, "").toString -// val (tempSave, useTempTable, dumpTableName, tempTableName) = { -// val (dump, dumpName) = useTable(DumpTableName) -// val (temp, tempName) = useTable(TempTableName) -// if (dump) { -// (false, dumpName, tempName) -// } else if (temp) { -// (true, dumpName, tempName) -// } else throw new Exception("invalid dump table name and temporary table name!") -// } - -// private def useTable(key: String): (Boolean, String) = { -// dataConnectorParam.config.get(key) match { -// case Some(name: String) => { -// name match { -// case TableNameRegex() => (true, name) -// case _ => (false, name) -// } -// } -// case _ => (false, "") -// } -// } - - def available(): Boolean = { -// kafkaStreamingDataConnector.available && cacheDataConnector.available - cacheDataConnector.available - } - - override def init(): Unit = { - cacheDataConnector.init - - val ds = kafkaStreamingDataConnector.stream match { - case Success(dstream) => dstream - case Failure(ex) => throw ex - } - - ds.foreachRDD((rdd, time) => { - val ms = time.milliseconds - - val dataInfoMap = DataInfo.cacheInfoList.map(_.defWrap).toMap + TimeStampInfo.wrap(ms) - - // parse each message - val valueMapRdd: RDD[Map[String, Any]] = rdd.flatMap { kv => - val msg = kv._2 - - val cacheExprValueMaps = ExprValueUtil.genExprValueMaps(Some(msg), ruleExprs.cacheExprs, constFinalExprValueMap) - val finalExprValueMaps = ExprValueUtil.updateExprValueMaps(ruleExprs.finalCacheExprs, cacheExprValueMaps) - - finalExprValueMaps.map { vm => - vm ++ dataInfoMap - } - } - - // generate DataFrame -// val df = genDataFrame(valueMapRdd) - - // save data frame - cacheDataConnector.saveData(valueMapRdd, ms) - }) - } - -// // generate DataFrame -// // maybe we can directly use def createDataFrame[A <: Product : TypeTag](rdd: RDD[A]): DataFrame -// // to avoid generate data type by myself, just translate each value into Product -// private def genDataFrame(rdd: RDD[Map[String, Any]]): DataFrame = { -// val fields = rdd.aggregate(Map[String, DataType]())( -// DataTypeCalculationUtil.sequenceDataTypeMap, DataTypeCalculationUtil.combineDataTypeMap -// ).toList.map(f => StructField(f._1, f._2)) -// val schema = StructType(fields) -// val datas: RDD[Row] = rdd.map { d => -// val values = fields.map { field => -// val StructField(k, dt, _, _) = field -// d.get(k) match { -// case Some(v) => v -// case _ => null -// } -// } -// Row(values: _*) -// } -// val df = sqlContext.createDataFrame(datas, schema) -// df -// } - - def metaData(): Try[Iterable[(String, String)]] = Try { - Map.empty[String, String] - } - - def data(): Try[RDD[(Product, (Map[String, Any], Map[String, Any]))]] = Try { - cacheDataConnector.readData match { - case Success(rdd) => { - rdd.flatMap { row => - // generate cache data -// val cacheExprValueMaps = ExprValueUtil.genExprValueMaps(Some(row), ruleExprs.cacheExprs, constFinalExprValueMap) -// val finalExprValueMaps = ExprValueUtil.updateExprValueMaps(ruleExprs.finalCacheExprs, cacheExprValueMaps) -// val finalExprValueMap = ruleExprs.finalCacheExprs.foldLeft(Map[String, Any]()) { (mp, expr) => -// mp + (expr._id -> row.get(expr._id)) -// } - val finalExprValueMap = ruleExprs.finalCacheExprs.flatMap { expr => - row.get(expr._id).flatMap { d => - Some((expr._id, d)) - } - }.toMap - - // data info -// val dataInfoMap: Map[String, Any] = DataInfo.cacheInfoList.map { info => -// try { -// (info.key -> row.getAs[info.T](info.key)) -// } catch { -// case e: Throwable => info.defWrap -// } -// }.toMap - val dataInfoMap: Map[String, Any] = DataInfo.cacheInfoList.map { info => - row.get(info.key) match { - case Some(d) => (info.key -> d) - case _ => info.defWrap - } - }.toMap - - val groupbyData: Seq[AnyRef] = ruleExprs.groupbyExprs.flatMap { expr => - expr.calculate(finalExprValueMap) match { - case Some(v) => Some(v.asInstanceOf[AnyRef]) - case _ => None - } - } - val key = toTuple(groupbyData) - - Some((key, (finalExprValueMap, dataInfoMap))) - } - } - case Failure(ex) => throw ex - } - } - - override def cleanOldData(): Unit = { - cacheDataConnector.cleanOldData - } - - override def updateOldData(t: Long, oldData: Iterable[Map[String, Any]]): Unit = { - if (dataConnectorParam.getMatchOnce) { - cacheDataConnector.updateOldData(t, oldData) - } - } - - override def updateAllOldData(oldRdd: RDD[Map[String, Any]]): Unit = { - if (dataConnectorParam.getMatchOnce) { - cacheDataConnector.updateAllOldData(oldRdd) - } - } - -// private def dbPrefix(): Boolean = { -// if (useTempTable) false else dumpDatabase.nonEmpty && !dumpDatabase.equals("default") -// } -// -// private def fullDumpTableName: String = if (dbPrefix) dumpTableName else s"${dumpDatabase}.${dumpTableName}" -// -// private def dumpTableExists(): Boolean = { -// (!dumpTableName.isEmpty) && { -// Try { -// if (dbPrefix) { -// sqlContext.tables(dumpDatabase).filter(dumpTableExistsSql).collect.size -// } else { -// sqlContext.tables().filter(dumpTableExistsSql).collect.size -// } -// } match { -// case Success(s) => s > 0 -// case _ => false -// } -// } -// } -// -// private def dumpTableExistsSql(): String = { -// s"tableName LIKE '${dumpTableName}'" -// } -// -// private def createDumpTableSql(df: DataFrame): Unit = { -//// df.schema. -//// s"CREATE TABLE IF NOT EXISTS ${fullDumpTableName} " -// } -// -// private def saveTempDataFrame(df: DataFrame): Unit = { -// df.registerTempTable() -// s"CREATE TABLE IF NOT EXISTS ${fullTempTableName} " -// } - - private def toTuple[A <: AnyRef](as: Seq[A]): Product = { - if (as.size > 0) { - val tupleClass = Class.forName("scala.Tuple" + as.size) - tupleClass.getConstructors.apply(0).newInstance(as: _*).asInstanceOf[Product] - } else None - } - -} diff --git a/measure/src/main/scala/org/apache/griffin/measure/connector/cache/CacheDataConnector.scala b/measure/src/main/scala/org/apache/griffin/measure/connector/cache/CacheDataConnector.scala new file mode 100644 index 000000000..1dfe8e2da --- /dev/null +++ b/measure/src/main/scala/org/apache/griffin/measure/connector/cache/CacheDataConnector.scala @@ -0,0 +1,33 @@ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ +package org.apache.griffin.measure.connector.cache + +import org.apache.griffin.measure.connector.DataConnector +import org.apache.spark.rdd.RDD + +import scala.util.Try + +trait CacheDataConnector extends DataConnector with DataCacheable with DataUpdatable { + + def saveData(rdd: RDD[Map[String, Any]], ms: Long): Unit + + def readData(): Try[RDD[Map[String, Any]]] + +} + diff --git a/measure/src/main/scala/org/apache/griffin/measure/connector/CacheDataConnector.scala b/measure/src/main/scala/org/apache/griffin/measure/connector/cache/DataCacheable.scala similarity index 87% rename from measure/src/main/scala/org/apache/griffin/measure/connector/CacheDataConnector.scala rename to measure/src/main/scala/org/apache/griffin/measure/connector/cache/DataCacheable.scala index 855d606b8..3659efd1c 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/connector/CacheDataConnector.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/connector/cache/DataCacheable.scala @@ -16,25 +16,16 @@ KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License. */ -package org.apache.griffin.measure.connector +package org.apache.griffin.measure.connector.cache import java.util.concurrent.atomic.AtomicLong -import org.apache.griffin.measure.cache.info.{InfoCacheInstance, TimeInfoCache, ZKInfoCache} -import org.apache.griffin.measure.utils.TimeUtil -import org.apache.spark.rdd.RDD -import org.apache.spark.sql.DataFrame +import org.apache.griffin.measure.cache.info.{InfoCacheInstance, TimeInfoCache} -import scala.util.Try - -trait CacheDataConnector extends DataConnector { +trait DataCacheable { protected val defCacheInfoPath = PathCounter.genPath - def saveData(rdd: RDD[Map[String, Any]], ms: Long): Unit - - def readData(): Try[RDD[Map[String, Any]]] - val cacheInfoPath: String val readyTimeInterval: Long val readyTimeDelay: Long diff --git a/measure/src/main/scala/org/apache/griffin/measure/connector/cache/DataUpdatable.scala b/measure/src/main/scala/org/apache/griffin/measure/connector/cache/DataUpdatable.scala new file mode 100644 index 000000000..07c818705 --- /dev/null +++ b/measure/src/main/scala/org/apache/griffin/measure/connector/cache/DataUpdatable.scala @@ -0,0 +1,30 @@ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ +package org.apache.griffin.measure.connector.cache + +import org.apache.spark.rdd.RDD + +trait DataUpdatable { + + def cleanOldData(): Unit = {} + + def updateOldData(t: Long, oldData: Iterable[Map[String, Any]]): Unit = {} + def updateAllOldData(oldRdd: RDD[Map[String, Any]]): Unit = {} + +} diff --git a/measure/src/main/scala/org/apache/griffin/measure/connector/DfCacheDataConnector.scala b/measure/src/main/scala/org/apache/griffin/measure/connector/cache/DfCacheDataConnector.scala similarity index 98% rename from measure/src/main/scala/org/apache/griffin/measure/connector/DfCacheDataConnector.scala rename to measure/src/main/scala/org/apache/griffin/measure/connector/cache/DfCacheDataConnector.scala index f2e49bacf..37ef51c7c 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/connector/DfCacheDataConnector.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/connector/cache/DfCacheDataConnector.scala @@ -16,12 +16,11 @@ KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License. */ -package org.apache.griffin.measure.connector +package org.apache.griffin.measure.connector.cache import java.util.concurrent.TimeUnit import org.apache.griffin.measure.cache.info.{InfoCacheInstance, TimeInfoCache} -import org.apache.griffin.measure.cache.lock.CacheLock import org.apache.griffin.measure.config.params.user.DataCacheParam import org.apache.griffin.measure.result.TimeStampInfo import org.apache.griffin.measure.rule.DataTypeCalculationUtil @@ -31,7 +30,7 @@ import org.apache.spark.sql.types._ import org.apache.spark.sql.{DataFrame, Row, SQLContext} import org.apache.spark.storage.StorageLevel -import scala.util.{Success, Try} +import scala.util.Try case class DfCacheDataConnector(sqlContext: SQLContext, dataCacheParam: DataCacheParam ) extends CacheDataConnector { @@ -76,6 +75,8 @@ case class DfCacheDataConnector(sqlContext: SQLContext, dataCacheParam: DataCach true } + def init(): Unit = {} + def saveData(rdd: RDD[Map[String, Any]], ms: Long): Unit = { val newCacheLocked = newCacheLock.lock(-1, TimeUnit.SECONDS) if (newCacheLocked) { @@ -242,7 +243,7 @@ case class DfCacheDataConnector(sqlContext: SQLContext, dataCacheParam: DataCach } -import scala.collection.mutable.{ArrayBuffer} +import scala.collection.mutable.ArrayBuffer object SparkRowFormatter { diff --git a/measure/src/main/scala/org/apache/griffin/measure/connector/HiveCacheDataConnector.scala b/measure/src/main/scala/org/apache/griffin/measure/connector/cache/HiveCacheDataConnector.scala similarity index 98% rename from measure/src/main/scala/org/apache/griffin/measure/connector/HiveCacheDataConnector.scala rename to measure/src/main/scala/org/apache/griffin/measure/connector/cache/HiveCacheDataConnector.scala index 6b83c0039..a9019a7e7 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/connector/HiveCacheDataConnector.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/connector/cache/HiveCacheDataConnector.scala @@ -16,7 +16,7 @@ KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License. */ -package org.apache.griffin.measure.connector +package org.apache.griffin.measure.connector.cache import java.util.concurrent.TimeUnit @@ -25,9 +25,8 @@ import org.apache.griffin.measure.config.params.user.DataCacheParam import org.apache.griffin.measure.result.TimeStampInfo import org.apache.griffin.measure.utils.{HdfsFileDumpUtil, HdfsUtil, JsonUtil, TimeUtil} import org.apache.spark.rdd.RDD -import org.apache.spark.sql.{DataFrame, SQLContext} +import org.apache.spark.sql.SQLContext import org.apache.spark.sql.hive.HiveContext -import org.apache.spark.storage.StorageLevel import scala.util.{Success, Try} diff --git a/measure/src/main/scala/org/apache/griffin/measure/connector/AvroBatchDataConnector.scala b/measure/src/main/scala/org/apache/griffin/measure/connector/direct/AvroDirectDataConnector.scala similarity index 89% rename from measure/src/main/scala/org/apache/griffin/measure/connector/AvroBatchDataConnector.scala rename to measure/src/main/scala/org/apache/griffin/measure/connector/direct/AvroDirectDataConnector.scala index b2f2f47a8..b45e5a988 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/connector/AvroBatchDataConnector.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/connector/direct/AvroDirectDataConnector.scala @@ -16,24 +16,20 @@ KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License. */ -package org.apache.griffin.measure.connector - -import org.apache.griffin.measure.rule.expr._ -import org.apache.spark.rdd.RDD -import org.apache.spark.sql.{DataFrame, Row, SQLContext} -import com.databricks.spark.avro._ - -import scala.util.{Success, Try} -import java.nio.file.{Files, Paths} +package org.apache.griffin.measure.connector.direct import org.apache.griffin.measure.result._ import org.apache.griffin.measure.rule.{ExprValueUtil, RuleExprs} import org.apache.griffin.measure.utils.HdfsUtil +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.SQLContext + +import scala.util.Try // data connector for avro file -case class AvroBatchDataConnector(sqlContext: SQLContext, config: Map[String, Any], - ruleExprs: RuleExprs, constFinalExprValueMap: Map[String, Any] - ) extends BatchDataConnector { +case class AvroDirectDataConnector(sqlContext: SQLContext, config: Map[String, Any], + ruleExprs: RuleExprs, constFinalExprValueMap: Map[String, Any] + ) extends DirectDataConnector { val FilePath = "file.path" val FileName = "file.name" @@ -55,6 +51,8 @@ case class AvroBatchDataConnector(sqlContext: SQLContext, config: Map[String, An (!concreteFileFullPath.isEmpty) && fileExist } + def init(): Unit = {} + def metaData(): Try[Iterable[(String, String)]] = { Try { val st = sqlContext.read.format("com.databricks.spark.avro").load(concreteFileFullPath).schema diff --git a/measure/src/main/scala/org/apache/griffin/measure/connector/BatchDataConnector.scala b/measure/src/main/scala/org/apache/griffin/measure/connector/direct/DirectDataConnector.scala similarity index 79% rename from measure/src/main/scala/org/apache/griffin/measure/connector/BatchDataConnector.scala rename to measure/src/main/scala/org/apache/griffin/measure/connector/direct/DirectDataConnector.scala index 656588c9c..ac1a79275 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/connector/BatchDataConnector.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/connector/direct/DirectDataConnector.scala @@ -16,14 +16,16 @@ KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License. */ -package org.apache.griffin.measure.connector +package org.apache.griffin.measure.connector.direct +import org.apache.griffin.measure.connector.DataConnector +import org.apache.griffin.measure.connector.cache.DataUpdatable import org.apache.spark.rdd.RDD import scala.util.Try -trait BatchDataConnector extends DataConnector { +trait DirectDataConnector extends DataConnector with DataUpdatable { def metaData(): Try[Iterable[(String, String)]] diff --git a/measure/src/main/scala/org/apache/griffin/measure/connector/HiveBatchDataConnector.scala b/measure/src/main/scala/org/apache/griffin/measure/connector/direct/HiveDirectDataConnector.scala similarity index 92% rename from measure/src/main/scala/org/apache/griffin/measure/connector/HiveBatchDataConnector.scala rename to measure/src/main/scala/org/apache/griffin/measure/connector/direct/HiveDirectDataConnector.scala index 59b3e6df1..7de2b02b6 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/connector/HiveBatchDataConnector.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/connector/direct/HiveDirectDataConnector.scala @@ -16,20 +16,19 @@ KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License. */ -package org.apache.griffin.measure.connector +package org.apache.griffin.measure.connector.direct import org.apache.griffin.measure.result._ import org.apache.griffin.measure.rule.{ExprValueUtil, RuleExprs} -import org.apache.griffin.measure.rule.expr._ import org.apache.spark.rdd.RDD -import org.apache.spark.sql.{DataFrame, Row, SQLContext} +import org.apache.spark.sql.SQLContext import scala.util.{Success, Try} // data connector for hive -case class HiveBatchDataConnector(sqlContext: SQLContext, config: Map[String, Any], - ruleExprs: RuleExprs, constFinalExprValueMap: Map[String, Any] - ) extends BatchDataConnector { +case class HiveDirectDataConnector(sqlContext: SQLContext, config: Map[String, Any], + ruleExprs: RuleExprs, constFinalExprValueMap: Map[String, Any] + ) extends DirectDataConnector { val Database = "database" val TableName = "table.name" @@ -61,6 +60,8 @@ case class HiveBatchDataConnector(sqlContext: SQLContext, config: Map[String, An } } + def init(): Unit = {} + def metaData(): Try[Iterable[(String, String)]] = { Try { val originRows = sqlContext.sql(metaDataSql).map(r => (r.getString(0), r.getString(1))).collect diff --git a/measure/src/main/scala/org/apache/griffin/measure/connector/direct/KafkaCacheDirectDataConnector.scala b/measure/src/main/scala/org/apache/griffin/measure/connector/direct/KafkaCacheDirectDataConnector.scala new file mode 100644 index 000000000..d2534cc77 --- /dev/null +++ b/measure/src/main/scala/org/apache/griffin/measure/connector/direct/KafkaCacheDirectDataConnector.scala @@ -0,0 +1,125 @@ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ +package org.apache.griffin.measure.connector.direct + +import org.apache.griffin.measure.config.params.user.DataConnectorParam +import org.apache.griffin.measure.connector.DataConnectorFactory +import org.apache.griffin.measure.connector.cache.CacheDataConnector +import org.apache.griffin.measure.connector.streaming.StreamingDataConnector +import org.apache.griffin.measure.result._ +import org.apache.griffin.measure.rule._ +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.SQLContext +import org.apache.spark.streaming.StreamingContext + +import scala.util.{Failure, Success, Try} + +case class KafkaCacheDirectDataConnector(@transient streamingDataConnectorTry: Try[StreamingDataConnector], + cacheDataConnectorTry: Try[CacheDataConnector], + dataConnectorParam: DataConnectorParam, + ruleExprs: RuleExprs, + constFinalExprValueMap: Map[String, Any] + ) extends StreamingCacheDirectDataConnector { + + val cacheDataConnector: CacheDataConnector = cacheDataConnectorTry match { + case Success(cntr) => cntr + case Failure(ex) => throw ex + } + @transient val streamingDataConnector: StreamingDataConnector = streamingDataConnectorTry match { + case Success(cntr) => cntr + case Failure(ex) => throw ex + } + + protected def transform(rdd: RDD[(streamingDataConnector.K, streamingDataConnector.V)], + ms: Long + ): RDD[Map[String, Any]] = { + val dataInfoMap = DataInfo.cacheInfoList.map(_.defWrap).toMap + TimeStampInfo.wrap(ms) + + rdd.flatMap { kv => + val msg = kv._2 + + val cacheExprValueMaps = ExprValueUtil.genExprValueMaps(Some(msg), ruleExprs.cacheExprs, constFinalExprValueMap) + val finalExprValueMaps = ExprValueUtil.updateExprValueMaps(ruleExprs.finalCacheExprs, cacheExprValueMaps) + + finalExprValueMaps.map { vm => + vm ++ dataInfoMap + } + } + } + + def metaData(): Try[Iterable[(String, String)]] = Try { + Map.empty[String, String] + } + + def data(): Try[RDD[(Product, (Map[String, Any], Map[String, Any]))]] = Try { + cacheDataConnector.readData match { + case Success(rdd) => { + rdd.flatMap { row => + val finalExprValueMap = ruleExprs.finalCacheExprs.flatMap { expr => + row.get(expr._id).flatMap { d => + Some((expr._id, d)) + } + }.toMap + + val dataInfoMap: Map[String, Any] = DataInfo.cacheInfoList.map { info => + row.get(info.key) match { + case Some(d) => (info.key -> d) + case _ => info.defWrap + } + }.toMap + + val groupbyData: Seq[AnyRef] = ruleExprs.groupbyExprs.flatMap { expr => + expr.calculate(finalExprValueMap) match { + case Some(v) => Some(v.asInstanceOf[AnyRef]) + case _ => None + } + } + val key = toTuple(groupbyData) + + Some((key, (finalExprValueMap, dataInfoMap))) + } + } + case Failure(ex) => throw ex + } + } + + override def cleanOldData(): Unit = { + cacheDataConnector.cleanOldData + } + + override def updateOldData(t: Long, oldData: Iterable[Map[String, Any]]): Unit = { + if (dataConnectorParam.getMatchOnce) { + cacheDataConnector.updateOldData(t, oldData) + } + } + + override def updateAllOldData(oldRdd: RDD[Map[String, Any]]): Unit = { + if (dataConnectorParam.getMatchOnce) { + cacheDataConnector.updateAllOldData(oldRdd) + } + } + + private def toTuple[A <: AnyRef](as: Seq[A]): Product = { + if (as.size > 0) { + val tupleClass = Class.forName("scala.Tuple" + as.size) + tupleClass.getConstructors.apply(0).newInstance(as: _*).asInstanceOf[Product] + } else None + } + +} diff --git a/measure/src/main/scala/org/apache/griffin/measure/connector/direct/StreamingCacheDirectDataConnector.scala b/measure/src/main/scala/org/apache/griffin/measure/connector/direct/StreamingCacheDirectDataConnector.scala new file mode 100644 index 000000000..87139d6a4 --- /dev/null +++ b/measure/src/main/scala/org/apache/griffin/measure/connector/direct/StreamingCacheDirectDataConnector.scala @@ -0,0 +1,60 @@ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ +package org.apache.griffin.measure.connector.direct + +import org.apache.griffin.measure.connector.cache.CacheDataConnector +import org.apache.griffin.measure.connector.streaming.StreamingDataConnector +import org.apache.griffin.measure.result.{DataInfo, TimeStampInfo} +import org.apache.griffin.measure.rule.ExprValueUtil +import org.apache.spark.rdd.RDD + +import scala.util.{Failure, Success} + +trait StreamingCacheDirectDataConnector extends DirectDataConnector { + + val cacheDataConnector: CacheDataConnector + @transient val streamingDataConnector: StreamingDataConnector + + def available(): Boolean = { + cacheDataConnector.available && streamingDataConnector.available + } + + def init(): Unit = { + cacheDataConnector.init + + val ds = streamingDataConnector.stream match { + case Success(dstream) => dstream + case Failure(ex) => throw ex + } + + ds.foreachRDD((rdd, time) => { + val ms = time.milliseconds + + val valueMapRdd = transform(rdd, ms) + + // save data frame + cacheDataConnector.saveData(valueMapRdd, ms) + }) + } + + protected def transform(rdd: RDD[(streamingDataConnector.K, streamingDataConnector.V)], + ms: Long + ): RDD[Map[String, Any]] + +} diff --git a/measure/src/main/scala/org/apache/griffin/measure/connector/KafkaStreamingDataConnector.scala b/measure/src/main/scala/org/apache/griffin/measure/connector/streaming/KafkaStreamingDataConnector.scala similarity index 75% rename from measure/src/main/scala/org/apache/griffin/measure/connector/KafkaStreamingDataConnector.scala rename to measure/src/main/scala/org/apache/griffin/measure/connector/streaming/KafkaStreamingDataConnector.scala index b7f932be7..fdd511d26 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/connector/KafkaStreamingDataConnector.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/connector/streaming/KafkaStreamingDataConnector.scala @@ -16,17 +16,20 @@ KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License. */ -package org.apache.griffin.measure.connector +package org.apache.griffin.measure.connector.streaming import kafka.serializer.Decoder +import org.apache.griffin.measure.connector.cache.{CacheDataConnector, DataCacheable} +import org.apache.griffin.measure.result.{DataInfo, TimeStampInfo} +import org.apache.griffin.measure.rule.{ExprValueUtil, RuleExprs} +import org.apache.spark.rdd.RDD import org.apache.spark.streaming.StreamingContext import org.apache.spark.streaming.dstream.InputDStream -import org.apache.spark.streaming.kafka.KafkaUtils -import scala.reflect.ClassTag -import scala.util.Try +import scala.util.{Failure, Success, Try} -abstract class KafkaStreamingDataConnector(ssc: StreamingContext, config: Map[String, Any] +abstract class KafkaStreamingDataConnector(@transient ssc: StreamingContext, + config: Map[String, Any] ) extends StreamingDataConnector { type KD <: Decoder[K] type VD <: Decoder[V] @@ -44,6 +47,8 @@ abstract class KafkaStreamingDataConnector(ssc: StreamingContext, config: Map[St true } + def init(): Unit = {} + def stream(): Try[InputDStream[(K, V)]] = Try { val topicSet = topics.split(",").toSet createDStream(topicSet) diff --git a/measure/src/main/scala/org/apache/griffin/measure/connector/StreamingDataConnector.scala b/measure/src/main/scala/org/apache/griffin/measure/connector/streaming/StreamingDataConnector.scala similarity index 84% rename from measure/src/main/scala/org/apache/griffin/measure/connector/StreamingDataConnector.scala rename to measure/src/main/scala/org/apache/griffin/measure/connector/streaming/StreamingDataConnector.scala index b597f6f4b..c37caac15 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/connector/StreamingDataConnector.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/connector/streaming/StreamingDataConnector.scala @@ -16,15 +16,15 @@ KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License. */ -package org.apache.griffin.measure.connector +package org.apache.griffin.measure.connector.streaming -import org.apache.griffin.measure.log.Loggable +import org.apache.griffin.measure.connector.DataConnector import org.apache.spark.streaming.dstream.InputDStream import scala.util.Try -trait StreamingDataConnector extends Loggable { +trait StreamingDataConnector extends DataConnector { type K type V diff --git a/measure/src/main/scala/org/apache/griffin/measure/utils/HdfsUtil.scala b/measure/src/main/scala/org/apache/griffin/measure/utils/HdfsUtil.scala index 8b8d15f85..c97a2112b 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/utils/HdfsUtil.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/utils/HdfsUtil.scala @@ -27,7 +27,7 @@ object HdfsUtil { private val conf = new Configuration() conf.set("dfs.support.append", "true") - conf.set("fs.defaultFS", "hdfs://localhost") // fixme + conf.set("fs.defaultFS", "hdfs://localhost") // debug @localhost private val dfs = FileSystem.get(conf) diff --git a/measure/src/test/scala/org/apache/griffin/measure/algo/batch/BatchAccuracyAlgoTest.scala b/measure/src/test/scala/org/apache/griffin/measure/algo/batch/BatchAccuracyAlgoTest.scala index 70d27f62b..b0618f15f 100644 --- a/measure/src/test/scala/org/apache/griffin/measure/algo/batch/BatchAccuracyAlgoTest.scala +++ b/measure/src/test/scala/org/apache/griffin/measure/algo/batch/BatchAccuracyAlgoTest.scala @@ -26,7 +26,8 @@ import org.apache.griffin.measure.config.params.env._ import org.apache.griffin.measure.config.params.user._ import org.apache.griffin.measure.config.reader._ import org.apache.griffin.measure.config.validator._ -import org.apache.griffin.measure.connector.{BatchDataConnector, DataConnector, DataConnectorFactory} +import org.apache.griffin.measure.connector.direct.DirectDataConnector +import org.apache.griffin.measure.connector.{DataConnector, DataConnectorFactory} import org.apache.griffin.measure.log.Loggable import org.apache.griffin.measure.rule.expr._ import org.apache.griffin.measure.rule.{ExprValueUtil, RuleAnalyzer, RuleFactory} @@ -119,8 +120,8 @@ class BatchAccuracyAlgoTest extends FunSuite with Matchers with BeforeAndAfter w } // data connector - val sourceDataConnector: BatchDataConnector = - DataConnectorFactory.getBatchDataConnector(sqlContext, userParam.sourceParam, + val sourceDataConnector: DirectDataConnector = + DataConnectorFactory.getDirectDataConnector(sqlContext, null, userParam.sourceParam, ruleAnalyzer.sourceRuleExprs, finalConstMap ) match { case Success(cntr) => { @@ -129,8 +130,8 @@ class BatchAccuracyAlgoTest extends FunSuite with Matchers with BeforeAndAfter w } case Failure(ex) => throw ex } - val targetDataConnector: BatchDataConnector = - DataConnectorFactory.getBatchDataConnector(sqlContext, userParam.targetParam, + val targetDataConnector: DirectDataConnector = + DataConnectorFactory.getDirectDataConnector(sqlContext, null, userParam.targetParam, ruleAnalyzer.targetRuleExprs, finalConstMap ) match { case Success(cntr) => { diff --git a/measure/src/test/scala/org/apache/griffin/measure/algo/batch/BatchProfileAlgoTest.scala b/measure/src/test/scala/org/apache/griffin/measure/algo/batch/BatchProfileAlgoTest.scala index a0fae5616..35624db2d 100644 --- a/measure/src/test/scala/org/apache/griffin/measure/algo/batch/BatchProfileAlgoTest.scala +++ b/measure/src/test/scala/org/apache/griffin/measure/algo/batch/BatchProfileAlgoTest.scala @@ -26,7 +26,8 @@ import org.apache.griffin.measure.config.params.env._ import org.apache.griffin.measure.config.params.user._ import org.apache.griffin.measure.config.reader._ import org.apache.griffin.measure.config.validator._ -import org.apache.griffin.measure.connector.{BatchDataConnector, DataConnector, DataConnectorFactory} +import org.apache.griffin.measure.connector.direct.DirectDataConnector +import org.apache.griffin.measure.connector.{DataConnector, DataConnectorFactory} import org.apache.griffin.measure.log.Loggable import org.apache.griffin.measure.rule.expr._ import org.apache.griffin.measure.rule.{ExprValueUtil, RuleAnalyzer, RuleFactory} @@ -118,8 +119,8 @@ class BatchProfileAlgoTest extends FunSuite with Matchers with BeforeAndAfter wi } // data connector - val sourceDataConnector: BatchDataConnector = - DataConnectorFactory.getBatchDataConnector(sqlContext, userParam.sourceParam, + val sourceDataConnector: DirectDataConnector = + DataConnectorFactory.getDirectDataConnector(sqlContext, null, userParam.sourceParam, ruleAnalyzer.sourceRuleExprs, finalConstMap ) match { case Success(cntr) => { diff --git a/measure/src/test/scala/org/apache/griffin/measure/algo/batch/DataFrameSaveTest.scala b/measure/src/test/scala/org/apache/griffin/measure/algo/batch/DataFrameSaveTest.scala index a597760a9..78b13fe5a 100644 --- a/measure/src/test/scala/org/apache/griffin/measure/algo/batch/DataFrameSaveTest.scala +++ b/measure/src/test/scala/org/apache/griffin/measure/algo/batch/DataFrameSaveTest.scala @@ -25,7 +25,8 @@ import org.apache.griffin.measure.config.params.env._ import org.apache.griffin.measure.config.params.user._ import org.apache.griffin.measure.config.reader._ import org.apache.griffin.measure.config.validator._ -import org.apache.griffin.measure.connector.{BatchDataConnector, DataConnectorFactory} +import org.apache.griffin.measure.connector.DataConnectorFactory +import org.apache.griffin.measure.connector.direct.DirectDataConnector import org.apache.griffin.measure.log.Loggable import org.apache.griffin.measure.rule.expr._ import org.apache.griffin.measure.rule.{ExprValueUtil, RuleAnalyzer, RuleFactory} @@ -117,8 +118,8 @@ class DataFrameSaveTest extends FunSuite with Matchers with BeforeAndAfter with } // data connector - val sourceDataConnector: BatchDataConnector = - DataConnectorFactory.getBatchDataConnector(sqlContext, userParam.sourceParam, + val sourceDataConnector: DirectDataConnector = + DataConnectorFactory.getDirectDataConnector(sqlContext, null, userParam.sourceParam, ruleAnalyzer.sourceRuleExprs, finalConstMap ) match { case Success(cntr) => { diff --git a/measure/src/test/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgoTest.scala b/measure/src/test/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgoTest.scala index 2aabf4a07..f22b55dc9 100644 --- a/measure/src/test/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgoTest.scala +++ b/measure/src/test/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgoTest.scala @@ -29,7 +29,8 @@ import org.apache.griffin.measure.config.params.env._ import org.apache.griffin.measure.config.params.user._ import org.apache.griffin.measure.config.reader._ import org.apache.griffin.measure.config.validator._ -import org.apache.griffin.measure.connector.{BatchDataConnector, DataConnector, DataConnectorFactory} +import org.apache.griffin.measure.connector.direct.DirectDataConnector +import org.apache.griffin.measure.connector.{DataConnector, DataConnectorFactory} import org.apache.griffin.measure.log.Loggable import org.apache.griffin.measure.persist.{Persist, PersistFactory, PersistType} import org.apache.griffin.measure.result._ @@ -157,8 +158,8 @@ class StreamingAccuracyAlgoTest extends FunSuite with Matchers with BeforeAndAft } // data connector - val sourceDataConnector: BatchDataConnector = - DataConnectorFactory.getDataConnector(sqlContext, ssc, userParam.sourceParam, + val sourceDataConnector: DirectDataConnector = + DataConnectorFactory.getDirectDataConnector(sqlContext, ssc, userParam.sourceParam, ruleAnalyzer.sourceRuleExprs, finalConstMap ) match { case Success(cntr) => { @@ -167,8 +168,8 @@ class StreamingAccuracyAlgoTest extends FunSuite with Matchers with BeforeAndAft } case Failure(ex) => throw ex } - val targetDataConnector: BatchDataConnector = - DataConnectorFactory.getDataConnector(sqlContext, ssc, userParam.targetParam, + val targetDataConnector: DirectDataConnector = + DataConnectorFactory.getDirectDataConnector(sqlContext, ssc, userParam.targetParam, ruleAnalyzer.targetRuleExprs, finalConstMap ) match { case Success(cntr) => { diff --git a/measure/src/test/scala/org/apache/griffin/measure/connector/ConnectorTest.scala b/measure/src/test/scala/org/apache/griffin/measure/connector/ConnectorTest.scala index 48771ab75..9cb6dfab3 100644 --- a/measure/src/test/scala/org/apache/griffin/measure/connector/ConnectorTest.scala +++ b/measure/src/test/scala/org/apache/griffin/measure/connector/ConnectorTest.scala @@ -93,221 +93,221 @@ class ConnectorTest extends FunSuite with Matchers with BeforeAndAfter { // } // } - test("connector") { - val kafkaConfig = Map[String, String]( - ("bootstrap.servers" -> "10.149.247.156:9092"), - ("group.id" -> "test"), - ("auto.offset.reset" -> "smallest"), - ("auto.commit.enable" -> "false") - ) - - val cacheConfig = Map[String, Any]( - ("table.name" -> "source"), - ("info.path" -> "src") - ) - - val cacheParam = DataCacheParam("df", cacheConfig, Nil) - - val config = Map[String, Any]( - ("kafka.config" -> kafkaConfig), - ("topics" -> "sss"), - ("key.type" -> "java.lang.String"), - ("value.type" -> "java.lang.String") - ) - - val infoCacheConfig = Map[String, Any]( - ("hosts" -> "localhost:2181"), - ("namespace" -> "griffin/infocache"), - ("lock.path" -> "lock"), - ("mode" -> "persist"), - ("init.clear" -> true), - ("close.clear" -> false) - ) - val name = "ttt" - - val icp = InfoCacheParam("zk", infoCacheConfig) - val icps = icp :: Nil - - InfoCacheInstance.initInstance(icps, name) - InfoCacheInstance.init - - - val connectorParam = DataConnectorParam("kafka", "0.8", config, cacheParam, false) - - val conf = new SparkConf().setMaster("local[*]").setAppName("ConnectorTest") - val sc = new SparkContext(conf) - sc.setLogLevel("WARN") - val sqlContext = new SQLContext(sc) - - val batchInterval = TimeUtil.milliseconds("2s") match { - case Some(interval) => Milliseconds(interval) - case _ => throw new Exception("invalid batch interval") - } - val ssc = new StreamingContext(sc, batchInterval) - ssc.checkpoint("/test/griffin/cp") - - val connector = DataConnectorFactory.getStreamingDataConnector(ssc, connectorParam) - - val streamingConnector = connector match { - case Success(c) => c - case _ => fail - } - - val dataCacheParam = connectorParam.cache - val cacheDataConnector = DataConnectorFactory.getCacheDataConnector(sqlContext, dataCacheParam) match { - case Success(cntr) => cntr - case Failure(ex) => throw ex - } - - /// - -// def genDataFrame(rdd: RDD[Map[String, Any]]): DataFrame = { -// val fields = rdd.aggregate(Map[String, DataType]())( -// DataTypeCalculationUtil.sequenceDataTypeMap, DataTypeCalculationUtil.combineDataTypeMap -// ).toList.map(f => StructField(f._1, f._2)) -// val schema = StructType(fields) -// val datas: RDD[Row] = rdd.map { d => -// val values = fields.map { field => -// val StructField(k, dt, _, _) = field -// d.get(k) match { -// case Some(v) => v -// case _ => null -// } -// } -// Row(values: _*) -// } -// val df = sqlContext.createDataFrame(datas, schema) -// df -// } - - val rules = "$source.json().name = 's2' AND $source.json().age = 32" - val ep = EvaluateRuleParam(1, rules) - - val ruleFactory = RuleFactory(ep) - val rule: StatementExpr = ruleFactory.generateRule() - val ruleAnalyzer: RuleAnalyzer = RuleAnalyzer(rule) - - val ruleExprs = ruleAnalyzer.sourceRuleExprs - val constFinalExprValueMap = Map[String, Any]() - - /// - - val ds = streamingConnector.stream match { - case Success(dstream) => dstream - case Failure(ex) => throw ex - } - - ds.foreachRDD((rdd, time) => { - val ms = time.milliseconds - - val data = rdd.collect - val str = data.mkString("\n") - - println(s"${ms}: \n${str}") - - val dataInfoMap = DataInfo.cacheInfoList.map(_.defWrap).toMap + TimeStampInfo.wrap(ms) - - // parse each message - val valueMapRdd: RDD[Map[String, Any]] = rdd.flatMap { kv => - val msg = kv._2 - - val cacheExprValueMaps = ExprValueUtil.genExprValueMaps(Some(msg), ruleExprs.cacheExprs, constFinalExprValueMap) - val finalExprValueMaps = ExprValueUtil.updateExprValueMaps(ruleExprs.finalCacheExprs, cacheExprValueMaps) - - // val sf = StructField("name", DataType.fromJson("string")) - // val schema: StructType = new StructType() - - finalExprValueMaps.map { vm => - vm ++ dataInfoMap - } - } - - val cnt = valueMapRdd.count - - val valueMaps = valueMapRdd.collect() - val valuestr = valueMaps.mkString("\n") - -// println(s"count: ${cnt}\n${valuestr}") - - // generate DataFrame -// val df = genDataFrame(valueMapRdd) -// df.show(10) - - // save data frame - cacheDataConnector.saveData(valueMapRdd, ms) - - // show data -// cacheDataConnector.readData() match { -// case Success(rdf) => rdf.show(10) -// case Failure(ex) => println(s"cache data error: ${ex.getMessage}") -// } -// -// cacheDataConnector.submitLastProcTime(ms) - }) - - // process thread - case class Process() extends Runnable { - val lock = InfoCacheInstance.genLock("process") - def run(): Unit = { - val locked = lock.lock(5, TimeUnit.SECONDS) - if (locked) { - try { - // show data - cacheDataConnector.readData() match { - case Success(rdd) => { - rdd.take(10).foreach(println) - println(s"count: ${rdd.count}") - } - case Failure(ex) => println(s"cache data error: ${ex.getMessage}") - } - -// val st = new Date().getTime - // get data -// val sourceData = sourceDataConnector.data match { -// case Success(dt) => dt -// case Failure(ex) => throw ex -// } -// val targetData = targetDataConnector.data match { -// case Success(dt) => dt -// case Failure(ex) => throw ex -// } -// -// // accuracy algorithm -// val (accuResult, missingRdd, matchedRdd) = accuracy(sourceData, targetData, ruleAnalyzer) -// -// println(accuResult) -// -// val et = new Date().getTime -// -// val missingRecords = missingRdd.map(record2String(_, ruleAnalyzer.sourceRuleExprs.persistExprs, ruleAnalyzer.targetRuleExprs.persistExprs)) - - } finally { - lock.unlock() - } - } - } - } - - val processInterval = TimeUtil.milliseconds("10s") match { - case Some(interval) => interval - case _ => throw new Exception("invalid batch interval") - } - val process = TimingProcess(processInterval, Process()) - - process.startup() - - - ssc.start() - ssc.awaitTermination() - ssc.stop(stopSparkContext=true, stopGracefully=true) - - // context stop - sc.stop - - InfoCacheInstance.close() - - process.shutdown() - - } +// test("connector") { + // val kafkaConfig = Map[String, String]( + // ("bootstrap.servers" -> "10.149.247.156:9092"), + // ("group.id" -> "test"), + // ("auto.offset.reset" -> "smallest"), + // ("auto.commit.enable" -> "false") + // ) + // + // val cacheConfig = Map[String, Any]( + // ("table.name" -> "source"), + // ("info.path" -> "src") + // ) + // + // val cacheParam = DataCacheParam("df", cacheConfig, Nil) + // + // val config = Map[String, Any]( + // ("kafka.config" -> kafkaConfig), + // ("topics" -> "sss"), + // ("key.type" -> "java.lang.String"), + // ("value.type" -> "java.lang.String") + // ) + // + // val infoCacheConfig = Map[String, Any]( + // ("hosts" -> "localhost:2181"), + // ("namespace" -> "griffin/infocache"), + // ("lock.path" -> "lock"), + // ("mode" -> "persist"), + // ("init.clear" -> true), + // ("close.clear" -> false) + // ) + // val name = "ttt" + // + // val icp = InfoCacheParam("zk", infoCacheConfig) + // val icps = icp :: Nil + // + // InfoCacheInstance.initInstance(icps, name) + // InfoCacheInstance.init + // + // + // val connectorParam = DataConnectorParam("kafka", "0.8", config, cacheParam, false) + // + // val conf = new SparkConf().setMaster("local[*]").setAppName("ConnectorTest") + // val sc = new SparkContext(conf) + // sc.setLogLevel("WARN") + // val sqlContext = new SQLContext(sc) + // + // val batchInterval = TimeUtil.milliseconds("2s") match { + // case Some(interval) => Milliseconds(interval) + // case _ => throw new Exception("invalid batch interval") + // } + // val ssc = new StreamingContext(sc, batchInterval) + // ssc.checkpoint("/test/griffin/cp") + // + // val connector = DataConnectorFactory.getStreamingDataConnector(ssc, connectorParam) + // + // val streamingConnector = connector match { + // case Success(c) => c + // case _ => fail + // } + // + // val dataCacheParam = connectorParam.cache + // val cacheDataConnector = DataConnectorFactory.getCacheDataConnector(sqlContext, dataCacheParam) match { + // case Success(cntr) => cntr + // case Failure(ex) => throw ex + // } + // + // /// + // + //// def genDataFrame(rdd: RDD[Map[String, Any]]): DataFrame = { + //// val fields = rdd.aggregate(Map[String, DataType]())( + //// DataTypeCalculationUtil.sequenceDataTypeMap, DataTypeCalculationUtil.combineDataTypeMap + //// ).toList.map(f => StructField(f._1, f._2)) + //// val schema = StructType(fields) + //// val datas: RDD[Row] = rdd.map { d => + //// val values = fields.map { field => + //// val StructField(k, dt, _, _) = field + //// d.get(k) match { + //// case Some(v) => v + //// case _ => null + //// } + //// } + //// Row(values: _*) + //// } + //// val df = sqlContext.createDataFrame(datas, schema) + //// df + //// } + // + // val rules = "$source.json().name = 's2' AND $source.json().age = 32" + // val ep = EvaluateRuleParam(1, rules) + // + // val ruleFactory = RuleFactory(ep) + // val rule: StatementExpr = ruleFactory.generateRule() + // val ruleAnalyzer: RuleAnalyzer = RuleAnalyzer(rule) + // + // val ruleExprs = ruleAnalyzer.sourceRuleExprs + // val constFinalExprValueMap = Map[String, Any]() + // + // /// + // + // val ds = streamingConnector.stream match { + // case Success(dstream) => dstream + // case Failure(ex) => throw ex + // } + // + // ds.foreachRDD((rdd, time) => { + // val ms = time.milliseconds + // + // val data = rdd.collect + // val str = data.mkString("\n") + // + // println(s"${ms}: \n${str}") + // + // val dataInfoMap = DataInfo.cacheInfoList.map(_.defWrap).toMap + TimeStampInfo.wrap(ms) + // + // // parse each message + // val valueMapRdd: RDD[Map[String, Any]] = rdd.flatMap { kv => + // val msg = kv._2 + // + // val cacheExprValueMaps = ExprValueUtil.genExprValueMaps(Some(msg), ruleExprs.cacheExprs, constFinalExprValueMap) + // val finalExprValueMaps = ExprValueUtil.updateExprValueMaps(ruleExprs.finalCacheExprs, cacheExprValueMaps) + // + // // val sf = StructField("name", DataType.fromJson("string")) + // // val schema: StructType = new StructType() + // + // finalExprValueMaps.map { vm => + // vm ++ dataInfoMap + // } + // } + // + // val cnt = valueMapRdd.count + // + // val valueMaps = valueMapRdd.collect() + // val valuestr = valueMaps.mkString("\n") + // + //// println(s"count: ${cnt}\n${valuestr}") + // + // // generate DataFrame + //// val df = genDataFrame(valueMapRdd) + //// df.show(10) + // + // // save data frame + // cacheDataConnector.saveData(valueMapRdd, ms) + // + // // show data + //// cacheDataConnector.readData() match { + //// case Success(rdf) => rdf.show(10) + //// case Failure(ex) => println(s"cache data error: ${ex.getMessage}") + //// } + //// + //// cacheDataConnector.submitLastProcTime(ms) + // }) + // + // // process thread + // case class Process() extends Runnable { + // val lock = InfoCacheInstance.genLock("process") + // def run(): Unit = { + // val locked = lock.lock(5, TimeUnit.SECONDS) + // if (locked) { + // try { + // // show data + // cacheDataConnector.readData() match { + // case Success(rdd) => { + // rdd.take(10).foreach(println) + // println(s"count: ${rdd.count}") + // } + // case Failure(ex) => println(s"cache data error: ${ex.getMessage}") + // } + // + //// val st = new Date().getTime + // // get data + //// val sourceData = sourceDataConnector.data match { + //// case Success(dt) => dt + //// case Failure(ex) => throw ex + //// } + //// val targetData = targetDataConnector.data match { + //// case Success(dt) => dt + //// case Failure(ex) => throw ex + //// } + //// + //// // accuracy algorithm + //// val (accuResult, missingRdd, matchedRdd) = accuracy(sourceData, targetData, ruleAnalyzer) + //// + //// println(accuResult) + //// + //// val et = new Date().getTime + //// + //// val missingRecords = missingRdd.map(record2String(_, ruleAnalyzer.sourceRuleExprs.persistExprs, ruleAnalyzer.targetRuleExprs.persistExprs)) + // + // } finally { + // lock.unlock() + // } + // } + // } + // } + // + // val processInterval = TimeUtil.milliseconds("10s") match { + // case Some(interval) => interval + // case _ => throw new Exception("invalid batch interval") + // } + // val process = TimingProcess(processInterval, Process()) + // + // process.startup() + // + // + // ssc.start() + // ssc.awaitTermination() + // ssc.stop(stopSparkContext=true, stopGracefully=true) + // + // // context stop + // sc.stop + // + // InfoCacheInstance.close() + // + // process.shutdown() + // + // } } From eedd6624cbee3915bb3b51cbd995dc8cf7a6ed21 Mon Sep 17 00:00:00 2001 From: Lionel Liu Date: Sun, 23 Jul 2017 16:53:09 +0800 Subject: [PATCH 028/111] v1 --- .../streaming/StreamingAccuracyAlgo.scala | 319 ++++++++++++------ .../streaming/StreamingAccuracyProcess.scala | 239 +++++++++++++ .../measure/cache/info/TimeInfoCache.scala | 23 ++ .../cache/result/CacheResultProcesser.scala | 4 +- .../connector/cache/DataCacheable.scala | 5 +- .../cache/DfCacheDataConnector.scala | 3 - .../cache/HiveCacheDataConnector.scala | 43 +-- .../streaming/StreamingAccuracyAlgoTest.scala | 143 +------- 8 files changed, 522 insertions(+), 257 deletions(-) create mode 100644 measure/src/main/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyProcess.scala diff --git a/measure/src/main/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgo.scala b/measure/src/main/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgo.scala index d974ebcbd..31ad18cde 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgo.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgo.scala @@ -23,12 +23,13 @@ import java.util.concurrent.{Executors, ThreadPoolExecutor, TimeUnit} import org.apache.griffin.measure.algo.AccuracyAlgo import org.apache.griffin.measure.algo.core.AccuracyCore -import org.apache.griffin.measure.cache.info.InfoCacheInstance +import org.apache.griffin.measure.cache.info.{InfoCacheInstance, TimeInfoCache} +import org.apache.griffin.measure.cache.result.CacheResultProcesser import org.apache.griffin.measure.config.params.AllParam import org.apache.griffin.measure.connector._ import org.apache.griffin.measure.connector.direct.DirectDataConnector -import org.apache.griffin.measure.persist.{Persist, PersistFactory} -import org.apache.griffin.measure.result.{AccuracyResult, TimeStampInfo} +import org.apache.griffin.measure.persist.{Persist, PersistFactory, PersistType} +import org.apache.griffin.measure.result.{AccuracyResult, MismatchInfo, TimeStampInfo} import org.apache.griffin.measure.rule.{ExprValueUtil, RuleAnalyzer, RuleFactory} import org.apache.griffin.measure.rule.expr._ import org.apache.griffin.measure.utils.TimeUtil @@ -62,17 +63,23 @@ case class StreamingAccuracyAlgo(allParam: AllParam) extends AccuracyAlgo { val ssc = new StreamingContext(sc, batchInterval) ssc.checkpoint(sparkParam.cpDir) + // init info cache instance + InfoCacheInstance.initInstance(envParam.infoCacheParams, metricName) + InfoCacheInstance.init + // start time val startTime = new Date().getTime() + val persistFactory = PersistFactory(envParam.persistParams, metricName) + // get persists to persist measure result - val persist: Persist = PersistFactory(envParam.persistParams, metricName).getPersists(startTime) + val appPersist: Persist = persistFactory.getPersists(startTime) // get spark application id val applicationId = sc.applicationId // persist start id - persist.start(applicationId) + appPersist.start(applicationId) // generate rule from rule param, generate rule analyzer val ruleFactory = RuleFactory(userParam.evaluateRuleParam) @@ -109,58 +116,175 @@ case class StreamingAccuracyAlgo(allParam: AllParam) extends AccuracyAlgo { case Failure(ex) => throw ex } - InfoCacheInstance.initInstance(envParam.infoCacheParams, metricName) - InfoCacheInstance.init + val cacheResultProcesser = CacheResultProcesser() + + // init data stream + sourceDataConnector.init() + targetDataConnector.init() + + val streamingAccuracyProcess = StreamingAccuracyProcess( + sourceDataConnector, targetDataConnector, + ruleAnalyzer, cacheResultProcesser, persistFactory, appPersist) // process thread - case class Process() extends Runnable { - val lock = InfoCacheInstance.genLock("process") - def run(): Unit = { - val locked = lock.lock(5, TimeUnit.SECONDS) - if (locked) { - try { - val st = new Date().getTime - // get data - val sourceData = sourceDataConnector.data match { - case Success(dt) => dt - case Failure(ex) => throw ex - } - val targetData = targetDataConnector.data match { - case Success(dt) => dt - case Failure(ex) => throw ex - } - - // accuracy algorithm - val (accuResult, missingRdd, matchedRdd) = accuracy(sourceData, targetData, ruleAnalyzer) - - println(accuResult) - - val et = new Date().getTime - - // persist time -// persist.log(et, s"calculation using time: ${et - st} ms") - - // persist result -// persist.result(et, accuResult) - val missingRecords = missingRdd.map(record2String(_, ruleAnalyzer.sourceRuleExprs.persistExprs, ruleAnalyzer.targetRuleExprs.persistExprs)) -// persist.missRecords(missingRecords) - -// val pet = new Date().getTime -// persist.log(pet, s"persist using time: ${pet - et} ms") - } finally { - lock.unlock() - } - } - } - } +// case class Process() extends Runnable { +// val lock = InfoCacheInstance.genLock("process") +// def run(): Unit = { +// val updateTime = new Date().getTime +// val locked = lock.lock(5, TimeUnit.SECONDS) +// if (locked) { +// try { +// val st = new Date().getTime +// +// TimeInfoCache.startTimeInfoCache +// +// // get data +// val sourceData = sourceDataConnector.data match { +// case Success(dt) => dt +// case Failure(ex) => throw ex +// } +// val targetData = targetDataConnector.data match { +// case Success(dt) => dt +// case Failure(ex) => throw ex +// } +// +// sourceData.cache +// targetData.cache +// +// println(s"sourceData.count: ${sourceData.count}") +// println(s"targetData.count: ${targetData.count}") +// +// // accuracy algorithm +// val (accuResult, missingRdd, matchedRdd) = accuracy(sourceData, targetData, ruleAnalyzer) +// println(s"accuResult: ${accuResult}") +// +// val ct = new Date().getTime +// appPersist.log(ct, s"calculation using time: ${ct - st} ms") +// +// sourceData.unpersist() +// targetData.unpersist() +// +// // result of every group +// val matchedGroups = reorgByTimeGroup(matchedRdd) +// val matchedGroupCount = matchedGroups.count +// println(s"===== matchedGroupCount: ${matchedGroupCount} =====") +// +// // get missing results +// val missingGroups = reorgByTimeGroup(missingRdd) +// val missingGroupCount = missingGroups.count +// println(s"===== missingGroupCount: ${missingGroupCount} =====") +// +// val groups = matchedGroups.cogroup(missingGroups) +// val groupCount = groups.count +// println(s"===== groupCount: ${groupCount} =====") +// +// val updateResults = groups.flatMap { group => +// val (t, (matchData, missData)) = group +// +// val matchSize = matchData.size +// val missSize = missData.size +// val res = AccuracyResult(missSize, matchSize + missSize) +// +// val updatedCacheResultOpt = cacheResultProcesser.genUpdateCacheResult(t, updateTime, res) +// +// updatedCacheResultOpt.flatMap { updatedCacheResult => +// Some((updatedCacheResult, (t, missData))) +// } +// } +// +// updateResults.cache +// +// val updateResultsPart = updateResults.map(_._1) +// val updateDataPart = updateResults.map(_._2) +// +// val updateResultsArray = updateResultsPart.collect() +// +// // update results cache (in driver) +// // collect action is traversable once action, it will make rdd updateResults empty +// updateResultsArray.foreach { updateResult => +// println(s"update result: ${updateResult}") +// cacheResultProcesser.update(updateResult) +// // persist result +// val persist: Persist = persistFactory.getPersists(updateResult.timeGroup) +// persist.result(updateTime, updateResult.result) +// } +// +// // record missing data and update old data (in executor) +// updateDataPart.foreach { grp => +// val (t, datas) = grp +// val persist: Persist = persistFactory.getPersists(t) +// // persist missing data +// val missStrings = datas.map { row => +// val (_, (value, info)) = row +// s"${value} [${info.getOrElse(MismatchInfo.key, "unknown")}]" +// } +// persist.records(missStrings, PersistType.MISS) +// // data connector update old data +// val dumpDatas = datas.map { r => +// val (_, (v, i)) = r +// v ++ i +// } +// +// println(t) +// dumpDatas.foreach(println) +// +// sourceDataConnector.updateOldData(t, dumpDatas) +// targetDataConnector.updateOldData(t, dumpDatas) // not correct +// } +// +// updateResults.unpersist() +// +// // dump missing rdd (this part not need for future version, only for current df cache data version) +// val dumpRdd: RDD[Map[String, Any]] = missingRdd.map { r => +// val (_, (v, i)) = r +// v ++ i +// } +// sourceDataConnector.updateAllOldData(dumpRdd) +// targetDataConnector.updateAllOldData(dumpRdd) // not correct +// +// TimeInfoCache.endTimeInfoCache +// +// val et = new Date().getTime +// appPersist.log(et, s"persist using time: ${et - ct} ms") +// +// } catch { +// case e: Throwable => error(s"process error: ${e.getMessage}") +// } finally { +// lock.unlock() +// } +// } +// } +// } val processInterval = TimeUtil.milliseconds(sparkParam.processInterval) match { case Some(interval) => interval case _ => throw new Exception("invalid batch interval") } - val process = TimingProcess(processInterval, Process()) + val process = TimingProcess(processInterval, streamingAccuracyProcess) + + // clean thread +// case class Clean() extends Runnable { +// val lock = InfoCacheInstance.genLock("clean") +// def run(): Unit = { +// val locked = lock.lock(5, TimeUnit.SECONDS) +// if (locked) { +// try { +// sourceDataConnector.cleanData +// targetDataConnector.cleanData +// } finally { +// lock.unlock() +// } +// } +// } +// } +// val cleanInterval = TimeUtil.milliseconds(cleanerParam.cleanInterval) match { +// case Some(interval) => interval +// case _ => throw new Exception("invalid batch interval") +// } +// val clean = TimingProcess(cleanInterval, Clean()) process.startup() +// clean.startup() ssc.start() ssc.awaitTermination() @@ -171,60 +295,61 @@ case class StreamingAccuracyAlgo(allParam: AllParam) extends AccuracyAlgo { InfoCacheInstance.close - persist.finish() + appPersist.finish() process.shutdown() +// clean.shutdown() } } // calculate accuracy between source data and target data - def accuracy(sourceData: RDD[(Product, (Map[String, Any], Map[String, Any]))], - targetData: RDD[(Product, (Map[String, Any], Map[String, Any]))], - ruleAnalyzer: RuleAnalyzer) = { - // 1. cogroup - val allKvs = sourceData.cogroup(targetData) - - // 2. accuracy calculation - val (accuResult, missingRdd, matchedRdd) = AccuracyCore.accuracy(allKvs, ruleAnalyzer) - - (accuResult, missingRdd, matchedRdd) - } - - // convert data into a string - def record2String(rec: (Product, (Map[String, Any], Map[String, Any])), sourcePersist: Iterable[Expr], targetPersist: Iterable[Expr]): String = { - val (key, (data, info)) = rec - val persistData = getPersistMap(data, sourcePersist) - val persistInfo = info.mapValues { value => - value match { - case vd: Map[String, Any] => getPersistMap(vd, targetPersist) - case v => v - } - }.map(identity) - s"${persistData} [${persistInfo}]" - } - - // get the expr value map of the persist expressions - private def getPersistMap(data: Map[String, Any], persist: Iterable[Expr]): Map[String, Any] = { - val persistMap = persist.map(e => (e._id, e.desc)).toMap - data.flatMap { pair => - val (k, v) = pair - persistMap.get(k) match { - case Some(d) => Some((d -> v)) - case _ => None - } - } - } - - def reorgByTimeGroup(rdd: RDD[(Product, (Map[String, Any], Map[String, Any]))] - ): RDD[(Long, (Product, (Map[String, Any], Map[String, Any])))] = { - rdd.flatMap { row => - val (key, (value, info)) = row - val b: Option[(Long, (Product, (Map[String, Any], Map[String, Any])))] = info.get(TimeStampInfo.key) match { - case Some(t: Long) => Some((t, row)) - case _ => None - } - b - } - } +// def accuracy(sourceData: RDD[(Product, (Map[String, Any], Map[String, Any]))], +// targetData: RDD[(Product, (Map[String, Any], Map[String, Any]))], +// ruleAnalyzer: RuleAnalyzer) = { +// // 1. cogroup +// val allKvs = sourceData.cogroup(targetData) +// +// // 2. accuracy calculation +// val (accuResult, missingRdd, matchedRdd) = AccuracyCore.accuracy(allKvs, ruleAnalyzer) +// +// (accuResult, missingRdd, matchedRdd) +// } + +// // convert data into a string +// def record2String(rec: (Product, (Map[String, Any], Map[String, Any])), sourcePersist: Iterable[Expr], targetPersist: Iterable[Expr]): String = { +// val (key, (data, info)) = rec +// val persistData = getPersistMap(data, sourcePersist) +// val persistInfo = info.mapValues { value => +// value match { +// case vd: Map[String, Any] => getPersistMap(vd, targetPersist) +// case v => v +// } +// }.map(identity) +// s"${persistData} [${persistInfo}]" +// } +// +// // get the expr value map of the persist expressions +// private def getPersistMap(data: Map[String, Any], persist: Iterable[Expr]): Map[String, Any] = { +// val persistMap = persist.map(e => (e._id, e.desc)).toMap +// data.flatMap { pair => +// val (k, v) = pair +// persistMap.get(k) match { +// case Some(d) => Some((d -> v)) +// case _ => None +// } +// } +// } + +// def reorgByTimeGroup(rdd: RDD[(Product, (Map[String, Any], Map[String, Any]))] +// ): RDD[(Long, (Product, (Map[String, Any], Map[String, Any])))] = { +// rdd.flatMap { row => +// val (key, (value, info)) = row +// val b: Option[(Long, (Product, (Map[String, Any], Map[String, Any])))] = info.get(TimeStampInfo.key) match { +// case Some(t: Long) => Some((t, row)) +// case _ => None +// } +// b +// } +// } } diff --git a/measure/src/main/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyProcess.scala b/measure/src/main/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyProcess.scala new file mode 100644 index 000000000..e3fabc1cf --- /dev/null +++ b/measure/src/main/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyProcess.scala @@ -0,0 +1,239 @@ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ +package org.apache.griffin.measure.algo.streaming + +import java.util.Date +import java.util.concurrent.TimeUnit + +import org.apache.griffin.measure.algo.core.AccuracyCore +import org.apache.griffin.measure.cache.info.{InfoCacheInstance, TimeInfoCache} +import org.apache.griffin.measure.cache.result.CacheResultProcesser +import org.apache.griffin.measure.connector.direct.DirectDataConnector +import org.apache.griffin.measure.log.Loggable +import org.apache.griffin.measure.persist._ +import org.apache.griffin.measure.result.{AccuracyResult, MismatchInfo, TimeStampInfo} +import org.apache.griffin.measure.rule._ +import org.apache.griffin.measure.rule.expr._ +import org.apache.spark.rdd.RDD + +import scala.util.{Failure, Success} + +case class StreamingAccuracyProcess(sourceDataConnector: DirectDataConnector, + targetDataConnector: DirectDataConnector, + ruleAnalyzer: RuleAnalyzer, + cacheResultProcesser: CacheResultProcesser, + persistFactory: PersistFactory, + appPersist: Persist + ) extends Runnable with Loggable { + + val lock = InfoCacheInstance.genLock("process") + + def run(): Unit = { +// println(s"cache count: ${cacheResultProcesser.cacheGroup.size}") + val updateTime = new Date().getTime + val locked = lock.lock(5, TimeUnit.SECONDS) + if (locked) { + try { + val st = new Date().getTime + + TimeInfoCache.startTimeInfoCache + + // get data + val sourceData = sourceDataConnector.data match { + case Success(dt) => dt + case Failure(ex) => throw ex + } + val targetData = targetDataConnector.data match { + case Success(dt) => dt + case Failure(ex) => throw ex + } + + sourceData.cache + targetData.cache + +// println(s"sourceData.count: ${sourceData.count}") +// println(s"targetData.count: ${targetData.count}") + + // accuracy algorithm + val (accuResult, missingRdd, matchedRdd) = accuracy(sourceData, targetData, ruleAnalyzer) +// println(s"accuResult: ${accuResult}") + + val ct = new Date().getTime + appPersist.log(ct, s"calculation using time: ${ct - st} ms") + + sourceData.unpersist() + targetData.unpersist() + + // result of every group + val matchedGroups = reorgByTimeGroup(matchedRdd) +// val matchedGroupCount = matchedGroups.count +// println(s"===== matchedGroupCount: ${matchedGroupCount} =====") + + // get missing results + val missingGroups = reorgByTimeGroup(missingRdd) +// val missingGroupCount = missingGroups.count +// println(s"===== missingGroupCount: ${missingGroupCount} =====") + + val groups = matchedGroups.cogroup(missingGroups) +// val groupCount = groups.count +// println(s"===== groupCount: ${groupCount} =====") + + val updateResults = groups.flatMap { group => + val (t, (matchData, missData)) = group + + val matchSize = matchData.size + val missSize = missData.size + val res = AccuracyResult(missSize, matchSize + missSize) + + val updatedCacheResultOpt = cacheResultProcesser.genUpdateCacheResult(t, updateTime, res) + + updatedCacheResultOpt.flatMap { updatedCacheResult => + Some((updatedCacheResult, (t, missData))) + } + } + + updateResults.cache + + val updateResultsPart = updateResults.map(_._1) + val updateDataPart = updateResults.map(_._2) + + val updateResultsArray = updateResultsPart.collect() + + // update results cache (in driver) + // collect action is traversable once action, it will make rdd updateResults empty + updateResultsArray.foreach { updateResult => +// println(s"update result: ${updateResult}") + cacheResultProcesser.update(updateResult) + // persist result + val persist: Persist = persistFactory.getPersists(updateResult.timeGroup) + persist.result(updateTime, updateResult.result) + } + + // record missing data and update old data (in executor) + updateDataPart.foreach { grp => + val (t, datas) = grp + val persist: Persist = persistFactory.getPersists(t) + // persist missing data + val missStrings = datas.map { row => + record2String(row, ruleAnalyzer.sourceRuleExprs.persistExprs, ruleAnalyzer.targetRuleExprs.persistExprs) + } + persist.records(missStrings, PersistType.MISS) + // data connector update old data + val dumpDatas = datas.map { r => + val (_, (v, i)) = r + v ++ i + } + +// println(t) +// dumpDatas.foreach(println) + + sourceDataConnector.updateOldData(t, dumpDatas) + targetDataConnector.updateOldData(t, dumpDatas) // not correct + } + + updateResults.unpersist() + + // dump missing rdd (this part not need for future version, only for current df cache data version) + val dumpRdd: RDD[Map[String, Any]] = missingRdd.map { r => + val (_, (v, i)) = r + v ++ i + } + sourceDataConnector.updateAllOldData(dumpRdd) + targetDataConnector.updateAllOldData(dumpRdd) // not correct + + TimeInfoCache.endTimeInfoCache + + // clean old data + cleanData() + + val et = new Date().getTime + appPersist.log(et, s"persist using time: ${et - ct} ms") + + } catch { + case e: Throwable => error(s"process error: ${e.getMessage}") + } finally { + lock.unlock() + } + } + } + + // clean old data and old result cache + def cleanData(): Unit = { + try { + sourceDataConnector.cleanOldData + targetDataConnector.cleanOldData + + val cleanTime = TimeInfoCache.getCleanTime + cacheResultProcesser.refresh(cleanTime) + } catch { + case e: Throwable => error(s"clean data error: ${e.getMessage}") + } + } + + // calculate accuracy between source data and target data + private def accuracy(sourceData: RDD[(Product, (Map[String, Any], Map[String, Any]))], + targetData: RDD[(Product, (Map[String, Any], Map[String, Any]))], + ruleAnalyzer: RuleAnalyzer) = { + // 1. cogroup + val allKvs = sourceData.cogroup(targetData) + + // 2. accuracy calculation + val (accuResult, missingRdd, matchedRdd) = AccuracyCore.accuracy(allKvs, ruleAnalyzer) + + (accuResult, missingRdd, matchedRdd) + } + + private def reorgByTimeGroup(rdd: RDD[(Product, (Map[String, Any], Map[String, Any]))] + ): RDD[(Long, (Product, (Map[String, Any], Map[String, Any])))] = { + rdd.flatMap { row => + val (key, (value, info)) = row + val b: Option[(Long, (Product, (Map[String, Any], Map[String, Any])))] = info.get(TimeStampInfo.key) match { + case Some(t: Long) => Some((t, row)) + case _ => None + } + b + } + } + + // convert data into a string + def record2String(rec: (Product, (Map[String, Any], Map[String, Any])), dataPersist: Iterable[Expr], infoPersist: Iterable[Expr]): String = { + val (key, (data, info)) = rec + val persistData = getPersistMap(data, dataPersist) + val persistInfo = info.mapValues { value => + value match { + case vd: Map[String, Any] => getPersistMap(vd, infoPersist) + case v => v + } + }.map(identity) + s"${persistData} [${persistInfo}]" + } + + // get the expr value map of the persist expressions + private def getPersistMap(data: Map[String, Any], persist: Iterable[Expr]): Map[String, Any] = { + val persistMap = persist.map(e => (e._id, e.desc)).toMap + data.flatMap { pair => + val (k, v) = pair + persistMap.get(k) match { + case Some(d) => Some((d -> v)) + case _ => None + } + } + } + +} diff --git a/measure/src/main/scala/org/apache/griffin/measure/cache/info/TimeInfoCache.scala b/measure/src/main/scala/org/apache/griffin/measure/cache/info/TimeInfoCache.scala index 0526e8011..ac0acff7f 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/cache/info/TimeInfoCache.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/cache/info/TimeInfoCache.scala @@ -37,6 +37,7 @@ object TimeInfoCache extends Loggable with Serializable { val finalCacheInfoPath = "info.final" val finalReadyTime = s"${finalCacheInfoPath}/${ReadyTime}" val finalLastProcTime = s"${finalCacheInfoPath}/${LastProcTime}" + val finalCleanTime = s"${finalCacheInfoPath}/${CleanTime}" def startTimeInfoCache(): Unit = { genFinalReadyTime @@ -46,8 +47,13 @@ object TimeInfoCache extends Loggable with Serializable { readTimeRange } + def getCleanTime(): Long = { + readCleanTime + } + def endTimeInfoCache: Unit = { genFinalLastProcTime + genFinalCleanTime } private def genFinalReadyTime(): Unit = { @@ -72,6 +78,17 @@ object TimeInfoCache extends Loggable with Serializable { InfoCacheInstance.cacheInfo(map) } + private def genFinalCleanTime(): Unit = { + val subPath = InfoCacheInstance.listKeys(infoPath) + val keys = subPath.map { p => s"${infoPath}/${p}/${CleanTime}" } + val result = InfoCacheInstance.readInfo(keys) + val time = keys.map { k => + getLong(result, k) + }.min + val map = Map[String, String]((finalCleanTime -> time.toString)) + InfoCacheInstance.cacheInfo(map) + } + private def readTimeRange(): (Long, Long) = { val map = InfoCacheInstance.readInfo(List(finalLastProcTime, finalReadyTime)) val lastProcTime = getLong(map, finalLastProcTime) @@ -79,6 +96,12 @@ object TimeInfoCache extends Loggable with Serializable { (lastProcTime + 1, curReadyTime) } + private def readCleanTime(): Long = { + val map = InfoCacheInstance.readInfo(List(finalCleanTime)) + val cleanTime = getLong(map, finalCleanTime) + cleanTime + } + private def getLong(map: Map[String, String], key: String): Long = { try { map.get(key) match { diff --git a/measure/src/main/scala/org/apache/griffin/measure/cache/result/CacheResultProcesser.scala b/measure/src/main/scala/org/apache/griffin/measure/cache/result/CacheResultProcesser.scala index a8251bae5..50d3adab6 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/cache/result/CacheResultProcesser.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/cache/result/CacheResultProcesser.scala @@ -25,7 +25,7 @@ import scala.collection.mutable.{Map => MutableMap} case class CacheResultProcesser() extends Loggable { - private val cacheGroup: MutableMap[Long, CacheResult] = MutableMap() + val cacheGroup: MutableMap[Long, CacheResult] = MutableMap() def genUpdateCacheResult(timeGroup: Long, updateTime: Long, result: Result): Option[CacheResult] = { cacheGroup.get(timeGroup) match { @@ -62,7 +62,7 @@ case class CacheResultProcesser() extends Loggable { val curCacheGroup = cacheGroup.toMap val deadCache = curCacheGroup.filter { pr => val (_, cr) = pr - cr.timeGroup <= overtime || cr.result.eventual() + cr.timeGroup < overtime || cr.result.eventual() } info(s"=== dead cache group count: ${deadCache.size} ===") deadCache.keySet.foreach(cacheGroup -= _) diff --git a/measure/src/main/scala/org/apache/griffin/measure/connector/cache/DataCacheable.scala b/measure/src/main/scala/org/apache/griffin/measure/connector/cache/DataCacheable.scala index 3659efd1c..2be87a67b 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/connector/cache/DataCacheable.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/connector/cache/DataCacheable.scala @@ -56,10 +56,13 @@ trait DataCacheable { } protected def submitCleanTime(ms: Long): Unit = { - val map = Map[String, String]((selfCleanTime -> ms.toString)) + val cleanTime = genCleanTime(ms) + val map = Map[String, String]((selfCleanTime -> cleanTime.toString)) InfoCacheInstance.cacheInfo(map) } + protected def genCleanTime(ms: Long): Long = ms + protected def readCleanTime(): Option[Long] = { val key = selfCleanTime val keys = key :: Nil diff --git a/measure/src/main/scala/org/apache/griffin/measure/connector/cache/DfCacheDataConnector.scala b/measure/src/main/scala/org/apache/griffin/measure/connector/cache/DfCacheDataConnector.scala index 37ef51c7c..f883b4fec 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/connector/cache/DfCacheDataConnector.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/connector/cache/DfCacheDataConnector.scala @@ -106,10 +106,8 @@ case class DfCacheDataConnector(sqlContext: SQLContext, dataCacheParam: DataCach def readData(): Try[RDD[Map[String, Any]]] = Try { val timeRange = TimeInfoCache.getTimeRange - println(s"timeRange: ${timeRange}") submitLastProcTime(timeRange._2) val reviseTimeRange = (timeRange._1 + deltaTimeRange._1, timeRange._2 + deltaTimeRange._2) - println(s"reviseTimeRange: ${reviseTimeRange}") // move new data frame to temp data frame val newCacheLocked = newCacheLock.lock(-1, TimeUnit.SECONDS) @@ -165,7 +163,6 @@ case class DfCacheDataConnector(sqlContext: SQLContext, dataCacheParam: DataCach try { val timeRange = TimeInfoCache.getTimeRange val reviseTimeRange = (timeRange._1 + deltaTimeRange._1, timeRange._2 + deltaTimeRange._2) - println(s"clean reviseTimeRange: ${reviseTimeRange}") oldDataFrame.unpersist() oldDataFrame = oldDataFrame.filter(s"${timeStampColumn} >= ${reviseTimeRange._1}") diff --git a/measure/src/main/scala/org/apache/griffin/measure/connector/cache/HiveCacheDataConnector.scala b/measure/src/main/scala/org/apache/griffin/measure/connector/cache/HiveCacheDataConnector.scala index a9019a7e7..e24118842 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/connector/cache/HiveCacheDataConnector.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/connector/cache/HiveCacheDataConnector.scala @@ -227,11 +227,10 @@ case class HiveCacheDataConnector(sqlContext: SQLContext, dataCacheParam: DataCa def readData(): Try[RDD[Map[String, Any]]] = Try { val timeRange = TimeInfoCache.getTimeRange - println(s"timeRange: ${timeRange}") submitLastProcTime(timeRange._2) val reviseTimeRange = (timeRange._1 + deltaTimeRange._1, timeRange._2 + deltaTimeRange._2) - println(s"reviseTimeRange: ${reviseTimeRange}") + submitCleanTime(reviseTimeRange._1) // read directly through partition info val partitionRange = getPartitionRange(reviseTimeRange._1, reviseTimeRange._2) @@ -251,24 +250,18 @@ case class HiveCacheDataConnector(sqlContext: SQLContext, dataCacheParam: DataCa val oldCacheLocked = oldCacheLock.lock(-1, TimeUnit.SECONDS) if (oldCacheLocked) { try { - val timeRange = TimeInfoCache.getTimeRange - val reviseTimeRange = (timeRange._1 + deltaTimeRange._1, timeRange._2 + deltaTimeRange._2) - println(s"clean reviseTimeRange: ${reviseTimeRange}") - - // drop partition - val lowerBound = getPartition(reviseTimeRange._1) - val sql = dropPartitionSql(concreteTableName, lowerBound) - sqlContext.sql(sql) - - // fixme: remove data -// readCleanTime match { -// case Some(ct) => { -// ; -// } -// case _ => { -// ; -// } -// } + val cleanTime = readCleanTime() + cleanTime match { + case Some(ct) => { + // drop partition + val bound = getPartition(ct) + val sql = dropPartitionSql(concreteTableName, bound) + sqlContext.sql(sql) + } + case _ => { + // do nothing + } + } } catch { case e: Throwable => error(s"clean old data error: ${e.getMessage}") } finally { @@ -304,6 +297,13 @@ case class HiveCacheDataConnector(sqlContext: SQLContext, dataCacheParam: DataCa } } + override protected def genCleanTime(ms: Long): Long = { + val minPartition = partition.last + val t1 = TimeUtil.timeToUnit(ms, minPartition._3) + val t2 = TimeUtil.timeFromUnit(t1, minPartition._3) + t2 + } + private def getPartition(ms: Long): List[(String, Any)] = { partition.map { p => val (name, _, unit) = p @@ -338,8 +338,9 @@ case class HiveCacheDataConnector(sqlContext: SQLContext, dataCacheParam: DataCa sql } private def dropPartitionSql(tbn: String, partition: List[(String, Any)]): String = { - val partitionSql = partition.map(ptn => (s"PARTITION ( `${ptn._1}` < ${ptn._2} ) ")).mkString(", ") + val partitionSql = partition.map(ptn => (s"PARTITION ( `${ptn._1}` < '${ptn._2}' ) ")).mkString(", ") val sql = s"""ALTER TABLE ${tbn} DROP ${partitionSql}""" + println(sql) sql } diff --git a/measure/src/test/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgoTest.scala b/measure/src/test/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgoTest.scala index f22b55dc9..704dc16aa 100644 --- a/measure/src/test/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgoTest.scala +++ b/measure/src/test/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgoTest.scala @@ -101,12 +101,12 @@ class StreamingAccuracyAlgoTest extends FunSuite with Matchers with BeforeAndAft // sqlContext = new SQLContext(sc) sqlContext = new HiveContext(sc) - val a = sqlContext.sql("select * from s1 limit 10") - // val a = sqlContext.sql("show tables") - a.show(10) - - val b = HdfsUtil.existPath("/griffin/streaming") - println(b) +// val a = sqlContext.sql("select * from s1 limit 10") +// // val a = sqlContext.sql("show tables") +// a.show(10) +// +// val b = HdfsUtil.existPath("/griffin/streaming") +// println(b) } test("algorithm") { @@ -188,138 +188,15 @@ class StreamingAccuracyAlgoTest extends FunSuite with Matchers with BeforeAndAft // my algo val algo = StreamingAccuracyAlgo(allParam) - // process thread - case class Process() extends Runnable { - val lock = InfoCacheInstance.genLock("process") - def run(): Unit = { - val updateTime = new Date().getTime - val locked = lock.lock(5, TimeUnit.SECONDS) - if (locked) { - try { - val st = new Date().getTime - - TimeInfoCache.startTimeInfoCache - - // get data - val sourceData = sourceDataConnector.data match { - case Success(dt) => dt - case Failure(ex) => throw ex - } - val targetData = targetDataConnector.data match { - case Success(dt) => dt - case Failure(ex) => throw ex - } - - println(s"sourceData.count: ${sourceData.count}") - println(s"targetData.count: ${targetData.count}") - - // accuracy algorithm - val (accuResult, missingRdd, matchedRdd) = algo.accuracy(sourceData, targetData, ruleAnalyzer) - println(accuResult) - - val ct = new Date().getTime - appPersist.log(ct, s"calculation using time: ${ct - st} ms") - - // result of every group - val matchedGroups = algo.reorgByTimeGroup(matchedRdd) - val matchedGroupCount = matchedGroups.count - println(s"===== matchedGroupCount: ${matchedGroupCount} =====") - - // get missing results - val missingGroups = algo.reorgByTimeGroup(missingRdd) - val missingGroupCount = missingGroups.count - println(s"===== missingGroupCount: ${missingGroupCount} =====") - - val groups = matchedGroups.cogroup(missingGroups) - val groupCount = groups.count - println(s"===== groupCount: ${groupCount} =====") - - val updateResults = groups.flatMap { group => - val (t, (matchData, missData)) = group - - val matchSize = matchData.size - val missSize = missData.size - val res = AccuracyResult(missSize, matchSize + missSize) - - val updatedCacheResultOpt = cacheResultProcesser.genUpdateCacheResult(t, updateTime, res) - - updatedCacheResultOpt.flatMap { updatedCacheResult => - Some((updatedCacheResult, (t, missData))) - } - } - - updateResults.cache - - val updateResultsPart = updateResults.map(_._1) - val updateDataPart = updateResults.map(_._2) - - val updateResultsArray = updateResultsPart.collect() - - // update results cache (in driver) - // collect action is traversable once action, it will make rdd updateResults empty - updateResultsArray.foreach { updateResult => - println(s"update result: ${updateResult}") - cacheResultProcesser.update(updateResult) - // persist result - val persist: Persist = persistFactory.getPersists(updateResult.timeGroup) - persist.result(updateTime, updateResult.result) - } - - // record missing data and update old data (in executor) - updateDataPart.foreach { grp => - val (t, datas) = grp - val persist: Persist = persistFactory.getPersists(t) - // persist missing data - val missStrings = datas.map { row => - val (_, (value, info)) = row - s"${value} [${info.getOrElse(MismatchInfo.key, "unknown")}]" - } - persist.records(missStrings, PersistType.MISS) - // data connector update old data - val dumpDatas = datas.map { r => - val (_, (v, i)) = r - v ++ i - } - - println(t) - dumpDatas.foreach(println) - - sourceDataConnector.updateOldData(t, dumpDatas) - targetDataConnector.updateOldData(t, dumpDatas) // not correct - } - - updateResults.unpersist() - - // dump missing rdd (this part not need for future version, only for current df cache data version) - val dumpRdd: RDD[Map[String, Any]] = missingRdd.map { r => - val (_, (v, i)) = r - v ++ i - } - sourceDataConnector.updateAllOldData(dumpRdd) - targetDataConnector.updateAllOldData(dumpRdd) // not correct - - // fixme: 3. refactor data connector module - - - TimeInfoCache.endTimeInfoCache - - val et = new Date().getTime - appPersist.log(et, s"persist using time: ${et - ct} ms") - - } catch { - case e: Throwable => error(s"process error: ${e.getMessage}") - } finally { - lock.unlock() - } - } - } - } + val streamingAccuracyProcess = StreamingAccuracyProcess( + sourceDataConnector, targetDataConnector, + ruleAnalyzer, cacheResultProcesser, persistFactory, appPersist) val processInterval = TimeUtil.milliseconds(sparkParam.processInterval) match { case Some(interval) => interval case _ => throw new Exception("invalid batch interval") } - val process = TimingProcess(processInterval, Process()) + val process = TimingProcess(processInterval, streamingAccuracyProcess) // clean thread // case class Clean() extends Runnable { From f2971cef6dabd47ad3b8f71a37ecc5d1ad651155 Mon Sep 17 00:00:00 2001 From: Lionel Liu Date: Sun, 23 Jul 2017 17:01:33 +0800 Subject: [PATCH 029/111] t1 --- .../griffin/measure/utils/HdfsUtil.scala | 2 +- .../algo/batch/BatchAccuracyAlgoTest.scala | 388 +++++++------- .../algo/batch/BatchProfileAlgoTest.scala | 346 ++++++------- .../algo/batch/DataFrameSaveTest.scala | 344 ++++++------ .../streaming/StreamingAccuracyAlgoTest.scala | 488 +++++++++--------- .../measure/cache/InfoCacheInstanceTest.scala | 156 +++--- .../measure/cache/ZKCacheLockTest.scala | 168 +++--- .../measure/cache/ZKInfoCacheTest.scala | 180 +++---- .../measure/rule/RuleAnalyzerTest.scala | 2 +- .../griffin/measure/rule/RuleParserTest.scala | 4 +- 10 files changed, 1039 insertions(+), 1039 deletions(-) diff --git a/measure/src/main/scala/org/apache/griffin/measure/utils/HdfsUtil.scala b/measure/src/main/scala/org/apache/griffin/measure/utils/HdfsUtil.scala index c97a2112b..3172420b8 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/utils/HdfsUtil.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/utils/HdfsUtil.scala @@ -27,7 +27,7 @@ object HdfsUtil { private val conf = new Configuration() conf.set("dfs.support.append", "true") - conf.set("fs.defaultFS", "hdfs://localhost") // debug @localhost +// conf.set("fs.defaultFS", "hdfs://localhost") // debug @localhost private val dfs = FileSystem.get(conf) diff --git a/measure/src/test/scala/org/apache/griffin/measure/algo/batch/BatchAccuracyAlgoTest.scala b/measure/src/test/scala/org/apache/griffin/measure/algo/batch/BatchAccuracyAlgoTest.scala index b0618f15f..6a603266f 100644 --- a/measure/src/test/scala/org/apache/griffin/measure/algo/batch/BatchAccuracyAlgoTest.scala +++ b/measure/src/test/scala/org/apache/griffin/measure/algo/batch/BatchAccuracyAlgoTest.scala @@ -1,198 +1,198 @@ -/* -Licensed to the Apache Software Foundation (ASF) under one -or more contributor license agreements. See the NOTICE file -distributed with this work for additional information -regarding copyright ownership. The ASF licenses this file -to you under the Apache License, Version 2.0 (the -"License"); you may not use this file except in compliance -with the License. You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, -software distributed under the License is distributed on an -"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -KIND, either express or implied. See the License for the -specific language governing permissions and limitations -under the License. -*/ -package org.apache.griffin.measure.algo - -import java.util.Date - -import org.apache.griffin.measure.algo.batch.BatchAccuracyAlgo -import org.apache.griffin.measure.config.params._ -import org.apache.griffin.measure.config.params.env._ -import org.apache.griffin.measure.config.params.user._ -import org.apache.griffin.measure.config.reader._ -import org.apache.griffin.measure.config.validator._ -import org.apache.griffin.measure.connector.direct.DirectDataConnector -import org.apache.griffin.measure.connector.{DataConnector, DataConnectorFactory} -import org.apache.griffin.measure.log.Loggable -import org.apache.griffin.measure.rule.expr._ -import org.apache.griffin.measure.rule.{ExprValueUtil, RuleAnalyzer, RuleFactory} -import org.apache.spark.rdd.RDD -import org.apache.spark.sql.SQLContext -import org.apache.spark.{SparkConf, SparkContext} -import org.junit.runner.RunWith -import org.scalatest.junit.JUnitRunner -import org.scalatest.{BeforeAndAfter, FunSuite, Matchers} - -import scala.util.{Failure, Success, Try} - - -@RunWith(classOf[JUnitRunner]) -class BatchAccuracyAlgoTest extends FunSuite with Matchers with BeforeAndAfter with Loggable { - - val envFile = "src/test/resources/env.json" - val confFile = "src/test/resources/config.json" -// val confFile = "{\"name\":\"accu1\",\"type\":\"accuracy\",\"source\":{\"type\":\"avro\",\"version\":\"1.7\",\"config\":{\"file.name\":\"src/test/resources/users_info_src.avro\"}},\"target\":{\"type\":\"avro\",\"version\":\"1.7\",\"config\":{\"file.name\":\"src/test/resources/users_info_target.avro\"}},\"evaluateRule\":{\"sampleRatio\":1,\"rules\":\"$source.user_id + 5 = $target.user_id + (2 + 3) AND $source.first_name + 12 = $target.first_name + (10 + 2) 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 AND (15 OR true) WHEN true AND $source.user_id > 10020\"}}" - val envFsType = "local" - val userFsType = "local" - - val args = Array(envFile, confFile) - - var sc: SparkContext = _ - var sqlContext: SQLContext = _ - - var allParam: AllParam = _ - - before { - // read param files - val envParam = readParamFile[EnvParam](envFile, envFsType) match { - case Success(p) => p - case Failure(ex) => { - error(ex.getMessage) - sys.exit(-2) - } - } - val userParam = readParamFile[UserParam](confFile, userFsType) match { - case Success(p) => p - case Failure(ex) => { - error(ex.getMessage) - sys.exit(-2) - } - } - allParam = AllParam(envParam, userParam) - - // validate param files - validateParams(allParam) match { - case Failure(ex) => { - error(ex.getMessage) - sys.exit(-3) - } - case _ => { - info("params validation pass") - } - } - - val metricName = userParam.name - val conf = new SparkConf().setMaster("local[*]").setAppName(metricName) - sc = new SparkContext(conf) - sqlContext = new SQLContext(sc) - } - - test("algorithm") { - Try { - val envParam = allParam.envParam - val userParam = allParam.userParam - - // start time - val startTime = new Date().getTime() - - // get spark application id - val applicationId = sc.applicationId - - // rules - val ruleFactory = RuleFactory(userParam.evaluateRuleParam) - val rule: StatementExpr = ruleFactory.generateRule() - val ruleAnalyzer: RuleAnalyzer = RuleAnalyzer(rule) - - ruleAnalyzer.constCacheExprs.foreach(println) - ruleAnalyzer.constFinalCacheExprs.foreach(println) - - // global cache data - val constExprValueMap = ExprValueUtil.genExprValueMaps(None, ruleAnalyzer.constCacheExprs, Map[String, Any]()) - val finalConstExprValueMap = ExprValueUtil.updateExprValueMaps(ruleAnalyzer.constFinalCacheExprs, constExprValueMap) - val finalConstMap = finalConstExprValueMap.headOption match { - case Some(m) => m - case _ => Map[String, Any]() - } - - // data connector - val sourceDataConnector: DirectDataConnector = - DataConnectorFactory.getDirectDataConnector(sqlContext, null, userParam.sourceParam, - ruleAnalyzer.sourceRuleExprs, finalConstMap - ) match { - case Success(cntr) => { - if (cntr.available) cntr - else throw new Exception("source data not available!") - } - case Failure(ex) => throw ex - } - val targetDataConnector: DirectDataConnector = - DataConnectorFactory.getDirectDataConnector(sqlContext, null, userParam.targetParam, - ruleAnalyzer.targetRuleExprs, finalConstMap - ) match { - case Success(cntr) => { - if (cntr.available) cntr - else throw new Exception("target data not available!") - } - case Failure(ex) => throw ex - } - - // get metadata -// val sourceMetaData: Iterable[(String, String)] = sourceDataConnector.metaData() match { -// case Success(md) => md +///* +//Licensed to the Apache Software Foundation (ASF) under one +//or more contributor license agreements. See the NOTICE file +//distributed with this work for additional information +//regarding copyright ownership. The ASF licenses this file +//to you under the Apache License, Version 2.0 (the +//"License"); you may not use this file except in compliance +//with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +//Unless required by applicable law or agreed to in writing, +//software distributed under the License is distributed on an +//"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +//KIND, either express or implied. See the License for the +//specific language governing permissions and limitations +//under the License. +//*/ +//package org.apache.griffin.measure.algo +// +//import java.util.Date +// +//import org.apache.griffin.measure.algo.batch.BatchAccuracyAlgo +//import org.apache.griffin.measure.config.params._ +//import org.apache.griffin.measure.config.params.env._ +//import org.apache.griffin.measure.config.params.user._ +//import org.apache.griffin.measure.config.reader._ +//import org.apache.griffin.measure.config.validator._ +//import org.apache.griffin.measure.connector.direct.DirectDataConnector +//import org.apache.griffin.measure.connector.{DataConnector, DataConnectorFactory} +//import org.apache.griffin.measure.log.Loggable +//import org.apache.griffin.measure.rule.expr._ +//import org.apache.griffin.measure.rule.{ExprValueUtil, RuleAnalyzer, RuleFactory} +//import org.apache.spark.rdd.RDD +//import org.apache.spark.sql.SQLContext +//import org.apache.spark.{SparkConf, SparkContext} +//import org.junit.runner.RunWith +//import org.scalatest.junit.JUnitRunner +//import org.scalatest.{BeforeAndAfter, FunSuite, Matchers} +// +//import scala.util.{Failure, Success, Try} +// +// +//@RunWith(classOf[JUnitRunner]) +//class BatchAccuracyAlgoTest extends FunSuite with Matchers with BeforeAndAfter with Loggable { +// +// val envFile = "src/test/resources/env.json" +// val confFile = "src/test/resources/config.json" +//// val confFile = "{\"name\":\"accu1\",\"type\":\"accuracy\",\"source\":{\"type\":\"avro\",\"version\":\"1.7\",\"config\":{\"file.name\":\"src/test/resources/users_info_src.avro\"}},\"target\":{\"type\":\"avro\",\"version\":\"1.7\",\"config\":{\"file.name\":\"src/test/resources/users_info_target.avro\"}},\"evaluateRule\":{\"sampleRatio\":1,\"rules\":\"$source.user_id + 5 = $target.user_id + (2 + 3) AND $source.first_name + 12 = $target.first_name + (10 + 2) 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 AND (15 OR true) WHEN true AND $source.user_id > 10020\"}}" +// val envFsType = "local" +// val userFsType = "local" +// +// val args = Array(envFile, confFile) +// +// var sc: SparkContext = _ +// var sqlContext: SQLContext = _ +// +// var allParam: AllParam = _ +// +// before { +// // read param files +// val envParam = readParamFile[EnvParam](envFile, envFsType) match { +// case Success(p) => p +// case Failure(ex) => { +// error(ex.getMessage) +// sys.exit(-2) +// } +// } +// val userParam = readParamFile[UserParam](confFile, userFsType) match { +// case Success(p) => p +// case Failure(ex) => { +// error(ex.getMessage) +// sys.exit(-2) +// } +// } +// allParam = AllParam(envParam, userParam) +// +// // validate param files +// validateParams(allParam) match { +// case Failure(ex) => { +// error(ex.getMessage) +// sys.exit(-3) +// } +// case _ => { +// info("params validation pass") +// } +// } +// +// val metricName = userParam.name +// val conf = new SparkConf().setMaster("local[*]").setAppName(metricName) +// sc = new SparkContext(conf) +// sqlContext = new SQLContext(sc) +// } +// +// test("algorithm") { +// Try { +// val envParam = allParam.envParam +// val userParam = allParam.userParam +// +// // start time +// val startTime = new Date().getTime() +// +// // get spark application id +// val applicationId = sc.applicationId +// +// // rules +// val ruleFactory = RuleFactory(userParam.evaluateRuleParam) +// val rule: StatementExpr = ruleFactory.generateRule() +// val ruleAnalyzer: RuleAnalyzer = RuleAnalyzer(rule) +// +// ruleAnalyzer.constCacheExprs.foreach(println) +// ruleAnalyzer.constFinalCacheExprs.foreach(println) +// +// // global cache data +// val constExprValueMap = ExprValueUtil.genExprValueMaps(None, ruleAnalyzer.constCacheExprs, Map[String, Any]()) +// val finalConstExprValueMap = ExprValueUtil.updateExprValueMaps(ruleAnalyzer.constFinalCacheExprs, constExprValueMap) +// val finalConstMap = finalConstExprValueMap.headOption match { +// case Some(m) => m +// case _ => Map[String, Any]() +// } +// +// // data connector +// val sourceDataConnector: DirectDataConnector = +// DataConnectorFactory.getDirectDataConnector(sqlContext, null, userParam.sourceParam, +// ruleAnalyzer.sourceRuleExprs, finalConstMap +// ) match { +// case Success(cntr) => { +// if (cntr.available) cntr +// else throw new Exception("source data not available!") +// } +// case Failure(ex) => throw ex +// } +// val targetDataConnector: DirectDataConnector = +// DataConnectorFactory.getDirectDataConnector(sqlContext, null, userParam.targetParam, +// ruleAnalyzer.targetRuleExprs, finalConstMap +// ) match { +// case Success(cntr) => { +// if (cntr.available) cntr +// else throw new Exception("target data not available!") +// } +// case Failure(ex) => throw ex +// } +// +// // get metadata +//// val sourceMetaData: Iterable[(String, String)] = sourceDataConnector.metaData() match { +//// case Success(md) => md +//// case Failure(ex) => throw ex +//// } +//// val targetMetaData: Iterable[(String, String)] = targetDataConnector.metaData() match { +//// case Success(md) => md +//// case Failure(ex) => throw ex +//// } +// +// // get data +// val sourceData: RDD[(Product, (Map[String, Any], Map[String, Any]))] = sourceDataConnector.data() match { +// case Success(dt) => dt // case Failure(ex) => throw ex // } -// val targetMetaData: Iterable[(String, String)] = targetDataConnector.metaData() match { -// case Success(md) => md +// val targetData: RDD[(Product, (Map[String, Any], Map[String, Any]))] = targetDataConnector.data() match { +// case Success(dt) => dt // case Failure(ex) => throw ex // } - - // get data - val sourceData: RDD[(Product, (Map[String, Any], Map[String, Any]))] = sourceDataConnector.data() match { - case Success(dt) => dt - case Failure(ex) => throw ex - } - val targetData: RDD[(Product, (Map[String, Any], Map[String, Any]))] = targetDataConnector.data() match { - case Success(dt) => dt - case Failure(ex) => throw ex - } - - // my algo - val algo = BatchAccuracyAlgo(allParam) - - // accuracy algorithm - val (accuResult, missingRdd, matchedRdd) = algo.accuracy(sourceData, targetData, ruleAnalyzer) - - println(s"match percentage: ${accuResult.matchPercentage}, total count: ${accuResult.total}") - - missingRdd.map(rec => algo.record2String(rec, ruleAnalyzer.sourceRuleExprs.persistExprs, ruleAnalyzer.targetRuleExprs.persistExprs)).foreach(println) - - // end time - val endTime = new Date().getTime - println(s"using time: ${endTime - startTime} ms") - } match { - case Failure(ex) => { - error(ex.getMessage) - sys.exit(-4) - } - case _ => { - info("calculation finished") - } - } - } - - private def readParamFile[T <: Param](file: String, fsType: String)(implicit m : Manifest[T]): Try[T] = { - val paramReader = ParamReaderFactory.getParamReader(file, fsType) - paramReader.readConfig[T] - } - - private def validateParams(allParam: AllParam): Try[Boolean] = { - val allParamValidator = AllParamValidator() - allParamValidator.validate(allParam) - } - -} +// +// // my algo +// val algo = BatchAccuracyAlgo(allParam) +// +// // accuracy algorithm +// val (accuResult, missingRdd, matchedRdd) = algo.accuracy(sourceData, targetData, ruleAnalyzer) +// +// println(s"match percentage: ${accuResult.matchPercentage}, total count: ${accuResult.total}") +// +// missingRdd.map(rec => algo.record2String(rec, ruleAnalyzer.sourceRuleExprs.persistExprs, ruleAnalyzer.targetRuleExprs.persistExprs)).foreach(println) +// +// // end time +// val endTime = new Date().getTime +// println(s"using time: ${endTime - startTime} ms") +// } match { +// case Failure(ex) => { +// error(ex.getMessage) +// sys.exit(-4) +// } +// case _ => { +// info("calculation finished") +// } +// } +// } +// +// private def readParamFile[T <: Param](file: String, fsType: String)(implicit m : Manifest[T]): Try[T] = { +// val paramReader = ParamReaderFactory.getParamReader(file, fsType) +// paramReader.readConfig[T] +// } +// +// private def validateParams(allParam: AllParam): Try[Boolean] = { +// val allParamValidator = AllParamValidator() +// allParamValidator.validate(allParam) +// } +// +//} diff --git a/measure/src/test/scala/org/apache/griffin/measure/algo/batch/BatchProfileAlgoTest.scala b/measure/src/test/scala/org/apache/griffin/measure/algo/batch/BatchProfileAlgoTest.scala index 35624db2d..e0f500a34 100644 --- a/measure/src/test/scala/org/apache/griffin/measure/algo/batch/BatchProfileAlgoTest.scala +++ b/measure/src/test/scala/org/apache/griffin/measure/algo/batch/BatchProfileAlgoTest.scala @@ -1,173 +1,173 @@ -/* -Licensed to the Apache Software Foundation (ASF) under one -or more contributor license agreements. See the NOTICE file -distributed with this work for additional information -regarding copyright ownership. The ASF licenses this file -to you under the Apache License, Version 2.0 (the -"License"); you may not use this file except in compliance -with the License. You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, -software distributed under the License is distributed on an -"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -KIND, either express or implied. See the License for the -specific language governing permissions and limitations -under the License. -*/ -package org.apache.griffin.measure.algo - -import java.util.Date - -import org.apache.griffin.measure.algo.batch.BatchProfileAlgo -import org.apache.griffin.measure.config.params._ -import org.apache.griffin.measure.config.params.env._ -import org.apache.griffin.measure.config.params.user._ -import org.apache.griffin.measure.config.reader._ -import org.apache.griffin.measure.config.validator._ -import org.apache.griffin.measure.connector.direct.DirectDataConnector -import org.apache.griffin.measure.connector.{DataConnector, DataConnectorFactory} -import org.apache.griffin.measure.log.Loggable -import org.apache.griffin.measure.rule.expr._ -import org.apache.griffin.measure.rule.{ExprValueUtil, RuleAnalyzer, RuleFactory} -import org.apache.spark.rdd.RDD -import org.apache.spark.sql.SQLContext -import org.apache.spark.{SparkConf, SparkContext} -import org.junit.runner.RunWith -import org.scalatest.junit.JUnitRunner -import org.scalatest.{BeforeAndAfter, FunSuite, Matchers} - -import scala.util.{Failure, Success, Try} - - -@RunWith(classOf[JUnitRunner]) -class BatchProfileAlgoTest extends FunSuite with Matchers with BeforeAndAfter with Loggable { - - val envFile = "src/test/resources/env.json" - val confFile = "src/test/resources/config-profile.json" - val envFsType = "local" - val userFsType = "local" - - val args = Array(envFile, confFile) - - var sc: SparkContext = _ - var sqlContext: SQLContext = _ - - var allParam: AllParam = _ - - before { - // read param files - val envParam = readParamFile[EnvParam](envFile, envFsType) match { - case Success(p) => p - case Failure(ex) => { - error(ex.getMessage) - sys.exit(-2) - } - } - val userParam = readParamFile[UserParam](confFile, userFsType) match { - case Success(p) => p - case Failure(ex) => { - error(ex.getMessage) - sys.exit(-2) - } - } - allParam = AllParam(envParam, userParam) - - // validate param files - validateParams(allParam) match { - case Failure(ex) => { - error(ex.getMessage) - sys.exit(-3) - } - case _ => { - info("params validation pass") - } - } - - val metricName = userParam.name - val conf = new SparkConf().setMaster("local[*]").setAppName(metricName) - sc = new SparkContext(conf) - sqlContext = new SQLContext(sc) - } - - test("algorithm") { - Try { - val envParam = allParam.envParam - val userParam = allParam.userParam - - // start time - val startTime = new Date().getTime() - - // get spark application id - val applicationId = sc.applicationId - - // rules - val ruleFactory = RuleFactory(userParam.evaluateRuleParam) - val rule: StatementExpr = ruleFactory.generateRule() - val ruleAnalyzer: RuleAnalyzer = RuleAnalyzer(rule) - - ruleAnalyzer.constCacheExprs.foreach(println) - ruleAnalyzer.constFinalCacheExprs.foreach(println) - - // global cache data - val constExprValueMap = ExprValueUtil.genExprValueMaps(None, ruleAnalyzer.constCacheExprs, Map[String, Any]()) - val finalConstExprValueMap = ExprValueUtil.updateExprValueMaps(ruleAnalyzer.constFinalCacheExprs, constExprValueMap) - val finalConstMap = finalConstExprValueMap.headOption match { - case Some(m) => m - case _ => Map[String, Any]() - } - - // data connector - val sourceDataConnector: DirectDataConnector = - DataConnectorFactory.getDirectDataConnector(sqlContext, null, userParam.sourceParam, - ruleAnalyzer.sourceRuleExprs, finalConstMap - ) match { - case Success(cntr) => { - if (cntr.available) cntr - else throw new Exception("source data not available!") - } - case Failure(ex) => throw ex - } - - // get data - val sourceData: RDD[(Product, (Map[String, Any], Map[String, Any]))] = sourceDataConnector.data() match { - case Success(dt) => dt - case Failure(ex) => throw ex - } - - // my algo - val algo = BatchProfileAlgo(allParam) - - // profile algorithm - val (profileResult, missingRdd, matchedRdd) = algo.profile(sourceData, ruleAnalyzer) - - println(s"match percentage: ${profileResult.matchPercentage}, match count: ${profileResult.matchCount}, total count: ${profileResult.totalCount}") - - matchedRdd.map(rec => algo.record2String(rec, ruleAnalyzer.sourceRuleExprs.persistExprs)).foreach(println) - - // end time - val endTime = new Date().getTime - println(s"using time: ${endTime - startTime} ms") - } match { - case Failure(ex) => { - error(ex.getMessage) - sys.exit(-4) - } - case _ => { - info("calculation finished") - } - } - } - - private def readParamFile[T <: Param](file: String, fsType: String)(implicit m : Manifest[T]): Try[T] = { - val paramReader = ParamReaderFactory.getParamReader(file, fsType) - paramReader.readConfig[T] - } - - private def validateParams(allParam: AllParam): Try[Boolean] = { - val allParamValidator = AllParamValidator() - allParamValidator.validate(allParam) - } - -} +///* +//Licensed to the Apache Software Foundation (ASF) under one +//or more contributor license agreements. See the NOTICE file +//distributed with this work for additional information +//regarding copyright ownership. The ASF licenses this file +//to you under the Apache License, Version 2.0 (the +//"License"); you may not use this file except in compliance +//with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +//Unless required by applicable law or agreed to in writing, +//software distributed under the License is distributed on an +//"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +//KIND, either express or implied. See the License for the +//specific language governing permissions and limitations +//under the License. +//*/ +//package org.apache.griffin.measure.algo +// +//import java.util.Date +// +//import org.apache.griffin.measure.algo.batch.BatchProfileAlgo +//import org.apache.griffin.measure.config.params._ +//import org.apache.griffin.measure.config.params.env._ +//import org.apache.griffin.measure.config.params.user._ +//import org.apache.griffin.measure.config.reader._ +//import org.apache.griffin.measure.config.validator._ +//import org.apache.griffin.measure.connector.direct.DirectDataConnector +//import org.apache.griffin.measure.connector.{DataConnector, DataConnectorFactory} +//import org.apache.griffin.measure.log.Loggable +//import org.apache.griffin.measure.rule.expr._ +//import org.apache.griffin.measure.rule.{ExprValueUtil, RuleAnalyzer, RuleFactory} +//import org.apache.spark.rdd.RDD +//import org.apache.spark.sql.SQLContext +//import org.apache.spark.{SparkConf, SparkContext} +//import org.junit.runner.RunWith +//import org.scalatest.junit.JUnitRunner +//import org.scalatest.{BeforeAndAfter, FunSuite, Matchers} +// +//import scala.util.{Failure, Success, Try} +// +// +//@RunWith(classOf[JUnitRunner]) +//class BatchProfileAlgoTest extends FunSuite with Matchers with BeforeAndAfter with Loggable { +// +// val envFile = "src/test/resources/env.json" +// val confFile = "src/test/resources/config-profile.json" +// val envFsType = "local" +// val userFsType = "local" +// +// val args = Array(envFile, confFile) +// +// var sc: SparkContext = _ +// var sqlContext: SQLContext = _ +// +// var allParam: AllParam = _ +// +// before { +// // read param files +// val envParam = readParamFile[EnvParam](envFile, envFsType) match { +// case Success(p) => p +// case Failure(ex) => { +// error(ex.getMessage) +// sys.exit(-2) +// } +// } +// val userParam = readParamFile[UserParam](confFile, userFsType) match { +// case Success(p) => p +// case Failure(ex) => { +// error(ex.getMessage) +// sys.exit(-2) +// } +// } +// allParam = AllParam(envParam, userParam) +// +// // validate param files +// validateParams(allParam) match { +// case Failure(ex) => { +// error(ex.getMessage) +// sys.exit(-3) +// } +// case _ => { +// info("params validation pass") +// } +// } +// +// val metricName = userParam.name +// val conf = new SparkConf().setMaster("local[*]").setAppName(metricName) +// sc = new SparkContext(conf) +// sqlContext = new SQLContext(sc) +// } +// +// test("algorithm") { +// Try { +// val envParam = allParam.envParam +// val userParam = allParam.userParam +// +// // start time +// val startTime = new Date().getTime() +// +// // get spark application id +// val applicationId = sc.applicationId +// +// // rules +// val ruleFactory = RuleFactory(userParam.evaluateRuleParam) +// val rule: StatementExpr = ruleFactory.generateRule() +// val ruleAnalyzer: RuleAnalyzer = RuleAnalyzer(rule) +// +// ruleAnalyzer.constCacheExprs.foreach(println) +// ruleAnalyzer.constFinalCacheExprs.foreach(println) +// +// // global cache data +// val constExprValueMap = ExprValueUtil.genExprValueMaps(None, ruleAnalyzer.constCacheExprs, Map[String, Any]()) +// val finalConstExprValueMap = ExprValueUtil.updateExprValueMaps(ruleAnalyzer.constFinalCacheExprs, constExprValueMap) +// val finalConstMap = finalConstExprValueMap.headOption match { +// case Some(m) => m +// case _ => Map[String, Any]() +// } +// +// // data connector +// val sourceDataConnector: DirectDataConnector = +// DataConnectorFactory.getDirectDataConnector(sqlContext, null, userParam.sourceParam, +// ruleAnalyzer.sourceRuleExprs, finalConstMap +// ) match { +// case Success(cntr) => { +// if (cntr.available) cntr +// else throw new Exception("source data not available!") +// } +// case Failure(ex) => throw ex +// } +// +// // get data +// val sourceData: RDD[(Product, (Map[String, Any], Map[String, Any]))] = sourceDataConnector.data() match { +// case Success(dt) => dt +// case Failure(ex) => throw ex +// } +// +// // my algo +// val algo = BatchProfileAlgo(allParam) +// +// // profile algorithm +// val (profileResult, missingRdd, matchedRdd) = algo.profile(sourceData, ruleAnalyzer) +// +// println(s"match percentage: ${profileResult.matchPercentage}, match count: ${profileResult.matchCount}, total count: ${profileResult.totalCount}") +// +// matchedRdd.map(rec => algo.record2String(rec, ruleAnalyzer.sourceRuleExprs.persistExprs)).foreach(println) +// +// // end time +// val endTime = new Date().getTime +// println(s"using time: ${endTime - startTime} ms") +// } match { +// case Failure(ex) => { +// error(ex.getMessage) +// sys.exit(-4) +// } +// case _ => { +// info("calculation finished") +// } +// } +// } +// +// private def readParamFile[T <: Param](file: String, fsType: String)(implicit m : Manifest[T]): Try[T] = { +// val paramReader = ParamReaderFactory.getParamReader(file, fsType) +// paramReader.readConfig[T] +// } +// +// private def validateParams(allParam: AllParam): Try[Boolean] = { +// val allParamValidator = AllParamValidator() +// allParamValidator.validate(allParam) +// } +// +//} diff --git a/measure/src/test/scala/org/apache/griffin/measure/algo/batch/DataFrameSaveTest.scala b/measure/src/test/scala/org/apache/griffin/measure/algo/batch/DataFrameSaveTest.scala index 78b13fe5a..a76712f76 100644 --- a/measure/src/test/scala/org/apache/griffin/measure/algo/batch/DataFrameSaveTest.scala +++ b/measure/src/test/scala/org/apache/griffin/measure/algo/batch/DataFrameSaveTest.scala @@ -1,172 +1,172 @@ -/* -Licensed to the Apache Software Foundation (ASF) under one -or more contributor license agreements. See the NOTICE file -distributed with this work for additional information -regarding copyright ownership. The ASF licenses this file -to you under the Apache License, Version 2.0 (the -"License"); you may not use this file except in compliance -with the License. You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, -software distributed under the License is distributed on an -"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -KIND, either express or implied. See the License for the -specific language governing permissions and limitations -under the License. -*/ -package org.apache.griffin.measure.algo.batch - -import java.util.Date - -import org.apache.griffin.measure.config.params._ -import org.apache.griffin.measure.config.params.env._ -import org.apache.griffin.measure.config.params.user._ -import org.apache.griffin.measure.config.reader._ -import org.apache.griffin.measure.config.validator._ -import org.apache.griffin.measure.connector.DataConnectorFactory -import org.apache.griffin.measure.connector.direct.DirectDataConnector -import org.apache.griffin.measure.log.Loggable -import org.apache.griffin.measure.rule.expr._ -import org.apache.griffin.measure.rule.{ExprValueUtil, RuleAnalyzer, RuleFactory} -import org.apache.spark.rdd.RDD -import org.apache.spark.sql.SQLContext -import org.apache.spark.{SparkConf, SparkContext} -import org.junit.runner.RunWith -import org.scalatest.junit.JUnitRunner -import org.scalatest.{BeforeAndAfter, FunSuite, Matchers} - -import scala.util.{Failure, Success, Try} - - -@RunWith(classOf[JUnitRunner]) -class DataFrameSaveTest extends FunSuite with Matchers with BeforeAndAfter with Loggable { - - val envFile = "src/test/resources/env.json" - val confFile = "src/test/resources/config-profile.json" - val envFsType = "local" - val userFsType = "local" - - val args = Array(envFile, confFile) - - var sc: SparkContext = _ - var sqlContext: SQLContext = _ - - var allParam: AllParam = _ - - before { - // read param files - val envParam = readParamFile[EnvParam](envFile, envFsType) match { - case Success(p) => p - case Failure(ex) => { - error(ex.getMessage) - sys.exit(-2) - } - } - val userParam = readParamFile[UserParam](confFile, userFsType) match { - case Success(p) => p - case Failure(ex) => { - error(ex.getMessage) - sys.exit(-2) - } - } - allParam = AllParam(envParam, userParam) - - // validate param files - validateParams(allParam) match { - case Failure(ex) => { - error(ex.getMessage) - sys.exit(-3) - } - case _ => { - info("params validation pass") - } - } - - val metricName = userParam.name - val conf = new SparkConf().setMaster("local[*]").setAppName(metricName) - sc = new SparkContext(conf) - sqlContext = new SQLContext(sc) - } - - test("algorithm") { - Try { - val envParam = allParam.envParam - val userParam = allParam.userParam - - // start time - val startTime = new Date().getTime() - - // get spark application id - val applicationId = sc.applicationId - - // rules - val ruleFactory = RuleFactory(userParam.evaluateRuleParam) - val rule: StatementExpr = ruleFactory.generateRule() - val ruleAnalyzer: RuleAnalyzer = RuleAnalyzer(rule) - - ruleAnalyzer.constCacheExprs.foreach(println) - ruleAnalyzer.constFinalCacheExprs.foreach(println) - - // global cache data - val constExprValueMap = ExprValueUtil.genExprValueMaps(None, ruleAnalyzer.constCacheExprs, Map[String, Any]()) - val finalConstExprValueMap = ExprValueUtil.updateExprValueMaps(ruleAnalyzer.constFinalCacheExprs, constExprValueMap) - val finalConstMap = finalConstExprValueMap.headOption match { - case Some(m) => m - case _ => Map[String, Any]() - } - - // data connector - val sourceDataConnector: DirectDataConnector = - DataConnectorFactory.getDirectDataConnector(sqlContext, null, userParam.sourceParam, - ruleAnalyzer.sourceRuleExprs, finalConstMap - ) match { - case Success(cntr) => { - if (cntr.available) cntr - else throw new Exception("source data not available!") - } - case Failure(ex) => throw ex - } - - // get data - val sourceData: RDD[(Product, (Map[String, Any], Map[String, Any]))] = sourceDataConnector.data() match { - case Success(dt) => dt - case Failure(ex) => throw ex - } - - // my algo - val algo = BatchProfileAlgo(allParam) - - // profile algorithm - val (profileResult, missingRdd, matchedRdd) = algo.profile(sourceData, ruleAnalyzer) - - println(s"match percentage: ${profileResult.matchPercentage}, match count: ${profileResult.matchCount}, total count: ${profileResult.totalCount}") - - matchedRdd.map(rec => algo.record2String(rec, ruleAnalyzer.sourceRuleExprs.persistExprs)).foreach(println) - - // end time - val endTime = new Date().getTime - println(s"using time: ${endTime - startTime} ms") - } match { - case Failure(ex) => { - error(ex.getMessage) - sys.exit(-4) - } - case _ => { - info("calculation finished") - } - } - } - - private def readParamFile[T <: Param](file: String, fsType: String)(implicit m : Manifest[T]): Try[T] = { - val paramReader = ParamReaderFactory.getParamReader(file, fsType) - paramReader.readConfig[T] - } - - private def validateParams(allParam: AllParam): Try[Boolean] = { - val allParamValidator = AllParamValidator() - allParamValidator.validate(allParam) - } - -} +///* +//Licensed to the Apache Software Foundation (ASF) under one +//or more contributor license agreements. See the NOTICE file +//distributed with this work for additional information +//regarding copyright ownership. The ASF licenses this file +//to you under the Apache License, Version 2.0 (the +//"License"); you may not use this file except in compliance +//with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +//Unless required by applicable law or agreed to in writing, +//software distributed under the License is distributed on an +//"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +//KIND, either express or implied. See the License for the +//specific language governing permissions and limitations +//under the License. +//*/ +//package org.apache.griffin.measure.algo.batch +// +//import java.util.Date +// +//import org.apache.griffin.measure.config.params._ +//import org.apache.griffin.measure.config.params.env._ +//import org.apache.griffin.measure.config.params.user._ +//import org.apache.griffin.measure.config.reader._ +//import org.apache.griffin.measure.config.validator._ +//import org.apache.griffin.measure.connector.DataConnectorFactory +//import org.apache.griffin.measure.connector.direct.DirectDataConnector +//import org.apache.griffin.measure.log.Loggable +//import org.apache.griffin.measure.rule.expr._ +//import org.apache.griffin.measure.rule.{ExprValueUtil, RuleAnalyzer, RuleFactory} +//import org.apache.spark.rdd.RDD +//import org.apache.spark.sql.SQLContext +//import org.apache.spark.{SparkConf, SparkContext} +//import org.junit.runner.RunWith +//import org.scalatest.junit.JUnitRunner +//import org.scalatest.{BeforeAndAfter, FunSuite, Matchers} +// +//import scala.util.{Failure, Success, Try} +// +// +//@RunWith(classOf[JUnitRunner]) +//class DataFrameSaveTest extends FunSuite with Matchers with BeforeAndAfter with Loggable { +// +// val envFile = "src/test/resources/env.json" +// val confFile = "src/test/resources/config-profile.json" +// val envFsType = "local" +// val userFsType = "local" +// +// val args = Array(envFile, confFile) +// +// var sc: SparkContext = _ +// var sqlContext: SQLContext = _ +// +// var allParam: AllParam = _ +// +// before { +// // read param files +// val envParam = readParamFile[EnvParam](envFile, envFsType) match { +// case Success(p) => p +// case Failure(ex) => { +// error(ex.getMessage) +// sys.exit(-2) +// } +// } +// val userParam = readParamFile[UserParam](confFile, userFsType) match { +// case Success(p) => p +// case Failure(ex) => { +// error(ex.getMessage) +// sys.exit(-2) +// } +// } +// allParam = AllParam(envParam, userParam) +// +// // validate param files +// validateParams(allParam) match { +// case Failure(ex) => { +// error(ex.getMessage) +// sys.exit(-3) +// } +// case _ => { +// info("params validation pass") +// } +// } +// +// val metricName = userParam.name +// val conf = new SparkConf().setMaster("local[*]").setAppName(metricName) +// sc = new SparkContext(conf) +// sqlContext = new SQLContext(sc) +// } +// +// test("algorithm") { +// Try { +// val envParam = allParam.envParam +// val userParam = allParam.userParam +// +// // start time +// val startTime = new Date().getTime() +// +// // get spark application id +// val applicationId = sc.applicationId +// +// // rules +// val ruleFactory = RuleFactory(userParam.evaluateRuleParam) +// val rule: StatementExpr = ruleFactory.generateRule() +// val ruleAnalyzer: RuleAnalyzer = RuleAnalyzer(rule) +// +// ruleAnalyzer.constCacheExprs.foreach(println) +// ruleAnalyzer.constFinalCacheExprs.foreach(println) +// +// // global cache data +// val constExprValueMap = ExprValueUtil.genExprValueMaps(None, ruleAnalyzer.constCacheExprs, Map[String, Any]()) +// val finalConstExprValueMap = ExprValueUtil.updateExprValueMaps(ruleAnalyzer.constFinalCacheExprs, constExprValueMap) +// val finalConstMap = finalConstExprValueMap.headOption match { +// case Some(m) => m +// case _ => Map[String, Any]() +// } +// +// // data connector +// val sourceDataConnector: DirectDataConnector = +// DataConnectorFactory.getDirectDataConnector(sqlContext, null, userParam.sourceParam, +// ruleAnalyzer.sourceRuleExprs, finalConstMap +// ) match { +// case Success(cntr) => { +// if (cntr.available) cntr +// else throw new Exception("source data not available!") +// } +// case Failure(ex) => throw ex +// } +// +// // get data +// val sourceData: RDD[(Product, (Map[String, Any], Map[String, Any]))] = sourceDataConnector.data() match { +// case Success(dt) => dt +// case Failure(ex) => throw ex +// } +// +// // my algo +// val algo = BatchProfileAlgo(allParam) +// +// // profile algorithm +// val (profileResult, missingRdd, matchedRdd) = algo.profile(sourceData, ruleAnalyzer) +// +// println(s"match percentage: ${profileResult.matchPercentage}, match count: ${profileResult.matchCount}, total count: ${profileResult.totalCount}") +// +// matchedRdd.map(rec => algo.record2String(rec, ruleAnalyzer.sourceRuleExprs.persistExprs)).foreach(println) +// +// // end time +// val endTime = new Date().getTime +// println(s"using time: ${endTime - startTime} ms") +// } match { +// case Failure(ex) => { +// error(ex.getMessage) +// sys.exit(-4) +// } +// case _ => { +// info("calculation finished") +// } +// } +// } +// +// private def readParamFile[T <: Param](file: String, fsType: String)(implicit m : Manifest[T]): Try[T] = { +// val paramReader = ParamReaderFactory.getParamReader(file, fsType) +// paramReader.readConfig[T] +// } +// +// private def validateParams(allParam: AllParam): Try[Boolean] = { +// val allParamValidator = AllParamValidator() +// allParamValidator.validate(allParam) +// } +// +//} diff --git a/measure/src/test/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgoTest.scala b/measure/src/test/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgoTest.scala index 704dc16aa..376eddad0 100644 --- a/measure/src/test/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgoTest.scala +++ b/measure/src/test/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgoTest.scala @@ -1,250 +1,250 @@ -/* -Licensed to the Apache Software Foundation (ASF) under one -or more contributor license agreements. See the NOTICE file -distributed with this work for additional information -regarding copyright ownership. The ASF licenses this file -to you under the Apache License, Version 2.0 (the -"License"); you may not use this file except in compliance -with the License. You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, -software distributed under the License is distributed on an -"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -KIND, either express or implied. See the License for the -specific language governing permissions and limitations -under the License. -*/ -package org.apache.griffin.measure.algo.streaming - -import java.util.Date -import java.util.concurrent.TimeUnit - -import org.apache.griffin.measure.algo.batch.BatchAccuracyAlgo -import org.apache.griffin.measure.cache.info.{InfoCacheInstance, TimeInfoCache} -import org.apache.griffin.measure.cache.result._ -import org.apache.griffin.measure.config.params._ -import org.apache.griffin.measure.config.params.env._ -import org.apache.griffin.measure.config.params.user._ -import org.apache.griffin.measure.config.reader._ -import org.apache.griffin.measure.config.validator._ -import org.apache.griffin.measure.connector.direct.DirectDataConnector -import org.apache.griffin.measure.connector.{DataConnector, DataConnectorFactory} -import org.apache.griffin.measure.log.Loggable -import org.apache.griffin.measure.persist.{Persist, PersistFactory, PersistType} -import org.apache.griffin.measure.result._ -import org.apache.griffin.measure.rule.expr._ -import org.apache.griffin.measure.rule.{ExprValueUtil, RuleAnalyzer, RuleFactory} -import org.apache.griffin.measure.utils.{HdfsUtil, TimeUtil} -import org.apache.spark.rdd.RDD -import org.apache.spark.sql.SQLContext -import org.apache.spark.sql.hive.HiveContext -import org.apache.spark.streaming.{Milliseconds, StreamingContext} -import org.apache.spark.{SparkConf, SparkContext} -import org.junit.runner.RunWith -import org.scalatest.junit.JUnitRunner -import org.scalatest.{BeforeAndAfter, FunSuite, Matchers} - -import scala.util.{Failure, Success, Try} - - -@RunWith(classOf[JUnitRunner]) -class StreamingAccuracyAlgoTest extends FunSuite with Matchers with BeforeAndAfter with Loggable { - - val envFile = "src/test/resources/env-streaming.json" - val confFile = "src/test/resources/config-streaming.json" - val envFsType = "local" - val userFsType = "local" - - val args = Array(envFile, confFile) - - var sc: SparkContext = _ - var sqlContext: SQLContext = _ -// val ssc: StreamingContext = _ - - var allParam: AllParam = _ - - before { - // read param files - val envParam = readParamFile[EnvParam](envFile, envFsType) match { - case Success(p) => p - case Failure(ex) => { - error(ex.getMessage) - sys.exit(-2) - } - } - val userParam = readParamFile[UserParam](confFile, userFsType) match { - case Success(p) => p - case Failure(ex) => { - error(ex.getMessage) - sys.exit(-2) - } - } - allParam = AllParam(envParam, userParam) - - // validate param files - validateParams(allParam) match { - case Failure(ex) => { - error(ex.getMessage) - sys.exit(-3) - } - case _ => { - info("params validation pass") - } - } - - val metricName = userParam.name - val conf = new SparkConf().setMaster("local[*]").setAppName(metricName) - sc = new SparkContext(conf) - sc.setLogLevel(envParam.sparkParam.logLevel) -// sqlContext = new SQLContext(sc) - sqlContext = new HiveContext(sc) - -// val a = sqlContext.sql("select * from s1 limit 10") -// // val a = sqlContext.sql("show tables") -// a.show(10) -// -// val b = HdfsUtil.existPath("/griffin/streaming") -// println(b) - } - - test("algorithm") { - val envParam = allParam.envParam - val userParam = allParam.userParam - val metricName = userParam.name - val sparkParam = envParam.sparkParam - val cleanerParam = envParam.cleanerParam - - val batchInterval = TimeUtil.milliseconds(sparkParam.batchInterval) match { - case Some(interval) => Milliseconds(interval) - case _ => throw new Exception("invalid batch interval") - } - val ssc = new StreamingContext(sc, batchInterval) - ssc.checkpoint(sparkParam.cpDir) - -// def createContext(): StreamingContext = { -// ; +///* +//Licensed to the Apache Software Foundation (ASF) under one +//or more contributor license agreements. See the NOTICE file +//distributed with this work for additional information +//regarding copyright ownership. The ASF licenses this file +//to you under the Apache License, Version 2.0 (the +//"License"); you may not use this file except in compliance +//with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +//Unless required by applicable law or agreed to in writing, +//software distributed under the License is distributed on an +//"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +//KIND, either express or implied. See the License for the +//specific language governing permissions and limitations +//under the License. +//*/ +//package org.apache.griffin.measure.algo.streaming +// +//import java.util.Date +//import java.util.concurrent.TimeUnit +// +//import org.apache.griffin.measure.algo.batch.BatchAccuracyAlgo +//import org.apache.griffin.measure.cache.info.{InfoCacheInstance, TimeInfoCache} +//import org.apache.griffin.measure.cache.result._ +//import org.apache.griffin.measure.config.params._ +//import org.apache.griffin.measure.config.params.env._ +//import org.apache.griffin.measure.config.params.user._ +//import org.apache.griffin.measure.config.reader._ +//import org.apache.griffin.measure.config.validator._ +//import org.apache.griffin.measure.connector.direct.DirectDataConnector +//import org.apache.griffin.measure.connector.{DataConnector, DataConnectorFactory} +//import org.apache.griffin.measure.log.Loggable +//import org.apache.griffin.measure.persist.{Persist, PersistFactory, PersistType} +//import org.apache.griffin.measure.result._ +//import org.apache.griffin.measure.rule.expr._ +//import org.apache.griffin.measure.rule.{ExprValueUtil, RuleAnalyzer, RuleFactory} +//import org.apache.griffin.measure.utils.{HdfsUtil, TimeUtil} +//import org.apache.spark.rdd.RDD +//import org.apache.spark.sql.SQLContext +//import org.apache.spark.sql.hive.HiveContext +//import org.apache.spark.streaming.{Milliseconds, StreamingContext} +//import org.apache.spark.{SparkConf, SparkContext} +//import org.junit.runner.RunWith +//import org.scalatest.junit.JUnitRunner +//import org.scalatest.{BeforeAndAfter, FunSuite, Matchers} +// +//import scala.util.{Failure, Success, Try} +// +// +//@RunWith(classOf[JUnitRunner]) +//class StreamingAccuracyAlgoTest extends FunSuite with Matchers with BeforeAndAfter with Loggable { +// +// val envFile = "src/test/resources/env-streaming.json" +// val confFile = "src/test/resources/config-streaming.json" +// val envFsType = "local" +// val userFsType = "local" +// +// val args = Array(envFile, confFile) +// +// var sc: SparkContext = _ +// var sqlContext: SQLContext = _ +//// val ssc: StreamingContext = _ +// +// var allParam: AllParam = _ +// +// before { +// // read param files +// val envParam = readParamFile[EnvParam](envFile, envFsType) match { +// case Success(p) => p +// case Failure(ex) => { +// error(ex.getMessage) +// sys.exit(-2) +// } // } - - // start time - val startTime = new Date().getTime() - - val persistFactory = PersistFactory(envParam.persistParams, metricName) - - // get persists to persist measure result - val appPersist: Persist = persistFactory.getPersists(startTime) - - // get spark application id - val applicationId = sc.applicationId - - // persist start id - appPersist.start(applicationId) - - InfoCacheInstance.initInstance(envParam.infoCacheParams, metricName) - InfoCacheInstance.init - - // generate rule from rule param, generate rule analyzer - val ruleFactory = RuleFactory(userParam.evaluateRuleParam) - val rule: StatementExpr = ruleFactory.generateRule() - val ruleAnalyzer: RuleAnalyzer = RuleAnalyzer(rule) - - // const expr value map - val constExprValueMap = ExprValueUtil.genExprValueMaps(None, ruleAnalyzer.constCacheExprs, Map[String, Any]()) - val finalConstExprValueMap = ExprValueUtil.updateExprValueMaps(ruleAnalyzer.constFinalCacheExprs, constExprValueMap) - val finalConstMap = finalConstExprValueMap.headOption match { - case Some(m) => m - case _ => Map[String, Any]() - } - - // data connector - val sourceDataConnector: DirectDataConnector = - DataConnectorFactory.getDirectDataConnector(sqlContext, ssc, userParam.sourceParam, - ruleAnalyzer.sourceRuleExprs, finalConstMap - ) match { - case Success(cntr) => { - if (cntr.available) cntr - else throw new Exception("source data connection error!") - } - case Failure(ex) => throw ex - } - val targetDataConnector: DirectDataConnector = - DataConnectorFactory.getDirectDataConnector(sqlContext, ssc, userParam.targetParam, - ruleAnalyzer.targetRuleExprs, finalConstMap - ) match { - case Success(cntr) => { - if (cntr.available) cntr - else throw new Exception("target data connection error!") - } - case Failure(ex) => throw ex - } - - val cacheResultProcesser = CacheResultProcesser() - - // init data stream - sourceDataConnector.init() - targetDataConnector.init() - - // my algo - val algo = StreamingAccuracyAlgo(allParam) - - val streamingAccuracyProcess = StreamingAccuracyProcess( - sourceDataConnector, targetDataConnector, - ruleAnalyzer, cacheResultProcesser, persistFactory, appPersist) - - val processInterval = TimeUtil.milliseconds(sparkParam.processInterval) match { - case Some(interval) => interval - case _ => throw new Exception("invalid batch interval") - } - val process = TimingProcess(processInterval, streamingAccuracyProcess) - - // clean thread -// case class Clean() extends Runnable { -// val lock = InfoCacheInstance.genLock("clean") -// def run(): Unit = { -// val locked = lock.lock(5, TimeUnit.SECONDS) -// if (locked) { -// try { -// sourceDataConnector.cleanData -// targetDataConnector.cleanData -// } finally { -// lock.unlock() -// } -// } +// val userParam = readParamFile[UserParam](confFile, userFsType) match { +// case Success(p) => p +// case Failure(ex) => { +// error(ex.getMessage) +// sys.exit(-2) // } // } -// val cleanInterval = TimeUtil.milliseconds(cleanerParam.cleanInterval) match { +// allParam = AllParam(envParam, userParam) +// +// // validate param files +// validateParams(allParam) match { +// case Failure(ex) => { +// error(ex.getMessage) +// sys.exit(-3) +// } +// case _ => { +// info("params validation pass") +// } +// } +// +// val metricName = userParam.name +// val conf = new SparkConf().setMaster("local[*]").setAppName(metricName) +// sc = new SparkContext(conf) +// sc.setLogLevel(envParam.sparkParam.logLevel) +//// sqlContext = new SQLContext(sc) +// sqlContext = new HiveContext(sc) +// +//// val a = sqlContext.sql("select * from s1 limit 10") +//// // val a = sqlContext.sql("show tables") +//// a.show(10) +//// +//// val b = HdfsUtil.existPath("/griffin/streaming") +//// println(b) +// } +// +// test("algorithm") { +// val envParam = allParam.envParam +// val userParam = allParam.userParam +// val metricName = userParam.name +// val sparkParam = envParam.sparkParam +// val cleanerParam = envParam.cleanerParam +// +// val batchInterval = TimeUtil.milliseconds(sparkParam.batchInterval) match { +// case Some(interval) => Milliseconds(interval) +// case _ => throw new Exception("invalid batch interval") +// } +// val ssc = new StreamingContext(sc, batchInterval) +// ssc.checkpoint(sparkParam.cpDir) +// +//// def createContext(): StreamingContext = { +//// ; +//// } +// +// // start time +// val startTime = new Date().getTime() +// +// val persistFactory = PersistFactory(envParam.persistParams, metricName) +// +// // get persists to persist measure result +// val appPersist: Persist = persistFactory.getPersists(startTime) +// +// // get spark application id +// val applicationId = sc.applicationId +// +// // persist start id +// appPersist.start(applicationId) +// +// InfoCacheInstance.initInstance(envParam.infoCacheParams, metricName) +// InfoCacheInstance.init +// +// // generate rule from rule param, generate rule analyzer +// val ruleFactory = RuleFactory(userParam.evaluateRuleParam) +// val rule: StatementExpr = ruleFactory.generateRule() +// val ruleAnalyzer: RuleAnalyzer = RuleAnalyzer(rule) +// +// // const expr value map +// val constExprValueMap = ExprValueUtil.genExprValueMaps(None, ruleAnalyzer.constCacheExprs, Map[String, Any]()) +// val finalConstExprValueMap = ExprValueUtil.updateExprValueMaps(ruleAnalyzer.constFinalCacheExprs, constExprValueMap) +// val finalConstMap = finalConstExprValueMap.headOption match { +// case Some(m) => m +// case _ => Map[String, Any]() +// } +// +// // data connector +// val sourceDataConnector: DirectDataConnector = +// DataConnectorFactory.getDirectDataConnector(sqlContext, ssc, userParam.sourceParam, +// ruleAnalyzer.sourceRuleExprs, finalConstMap +// ) match { +// case Success(cntr) => { +// if (cntr.available) cntr +// else throw new Exception("source data connection error!") +// } +// case Failure(ex) => throw ex +// } +// val targetDataConnector: DirectDataConnector = +// DataConnectorFactory.getDirectDataConnector(sqlContext, ssc, userParam.targetParam, +// ruleAnalyzer.targetRuleExprs, finalConstMap +// ) match { +// case Success(cntr) => { +// if (cntr.available) cntr +// else throw new Exception("target data connection error!") +// } +// case Failure(ex) => throw ex +// } +// +// val cacheResultProcesser = CacheResultProcesser() +// +// // init data stream +// sourceDataConnector.init() +// targetDataConnector.init() +// +// // my algo +// val algo = StreamingAccuracyAlgo(allParam) +// +// val streamingAccuracyProcess = StreamingAccuracyProcess( +// sourceDataConnector, targetDataConnector, +// ruleAnalyzer, cacheResultProcesser, persistFactory, appPersist) +// +// val processInterval = TimeUtil.milliseconds(sparkParam.processInterval) match { // case Some(interval) => interval // case _ => throw new Exception("invalid batch interval") // } -// val clean = TimingProcess(cleanInterval, Clean()) - - process.startup() -// clean.startup() - - ssc.start() - ssc.awaitTermination() - ssc.stop(stopSparkContext=true, stopGracefully=true) - - // context stop - sc.stop - - InfoCacheInstance.close - - appPersist.finish() - - process.shutdown() -// clean.shutdown() - } - - private def readParamFile[T <: Param](file: String, fsType: String)(implicit m : Manifest[T]): Try[T] = { - val paramReader = ParamReaderFactory.getParamReader(file, fsType) - paramReader.readConfig[T] - } - - private def validateParams(allParam: AllParam): Try[Boolean] = { - val allParamValidator = AllParamValidator() - allParamValidator.validate(allParam) - } - -} +// val process = TimingProcess(processInterval, streamingAccuracyProcess) +// +// // clean thread +//// case class Clean() extends Runnable { +//// val lock = InfoCacheInstance.genLock("clean") +//// def run(): Unit = { +//// val locked = lock.lock(5, TimeUnit.SECONDS) +//// if (locked) { +//// try { +//// sourceDataConnector.cleanData +//// targetDataConnector.cleanData +//// } finally { +//// lock.unlock() +//// } +//// } +//// } +//// } +//// val cleanInterval = TimeUtil.milliseconds(cleanerParam.cleanInterval) match { +//// case Some(interval) => interval +//// case _ => throw new Exception("invalid batch interval") +//// } +//// val clean = TimingProcess(cleanInterval, Clean()) +// +// process.startup() +//// clean.startup() +// +// ssc.start() +// ssc.awaitTermination() +// ssc.stop(stopSparkContext=true, stopGracefully=true) +// +// // context stop +// sc.stop +// +// InfoCacheInstance.close +// +// appPersist.finish() +// +// process.shutdown() +//// clean.shutdown() +// } +// +// private def readParamFile[T <: Param](file: String, fsType: String)(implicit m : Manifest[T]): Try[T] = { +// val paramReader = ParamReaderFactory.getParamReader(file, fsType) +// paramReader.readConfig[T] +// } +// +// private def validateParams(allParam: AllParam): Try[Boolean] = { +// val allParamValidator = AllParamValidator() +// allParamValidator.validate(allParam) +// } +// +//} diff --git a/measure/src/test/scala/org/apache/griffin/measure/cache/InfoCacheInstanceTest.scala b/measure/src/test/scala/org/apache/griffin/measure/cache/InfoCacheInstanceTest.scala index f754e3fb7..fc42d43bb 100644 --- a/measure/src/test/scala/org/apache/griffin/measure/cache/InfoCacheInstanceTest.scala +++ b/measure/src/test/scala/org/apache/griffin/measure/cache/InfoCacheInstanceTest.scala @@ -1,78 +1,78 @@ -/* -Licensed to the Apache Software Foundation (ASF) under one -or more contributor license agreements. See the NOTICE file -distributed with this work for additional information -regarding copyright ownership. The ASF licenses this file -to you under the Apache License, Version 2.0 (the -"License"); you may not use this file except in compliance -with the License. You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, -software distributed under the License is distributed on an -"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -KIND, either express or implied. See the License for the -specific language governing permissions and limitations -under the License. -*/ -package org.apache.griffin.measure.cache - -import java.util.Date -import java.util.concurrent.{Executors, ThreadPoolExecutor, TimeUnit} - -import org.apache.curator.framework.recipes.locks.InterProcessMutex -import org.apache.curator.framework.{CuratorFramework, CuratorFrameworkFactory} -import org.apache.curator.retry.ExponentialBackoffRetry -import org.apache.griffin.measure.cache.info.InfoCacheInstance -import org.apache.griffin.measure.config.params.env.InfoCacheParam -import org.junit.runner.RunWith -import org.scalatest.junit.JUnitRunner -import org.scalatest.{BeforeAndAfter, FunSuite, Matchers} - -import scala.util.{Failure, Try} - -@RunWith(classOf[JUnitRunner]) -class InfoCacheInstanceTest extends FunSuite with Matchers with BeforeAndAfter { - - val map = Map[String, Any]( - ("hosts" -> "localhost:2181"), - ("namespace" -> "griffin/infocache"), - ("lock.path" -> "lock"), - ("mode" -> "persist"), - ("init.clear" -> true), - ("close.clear" -> false) - ) - val name = "ttt" - - val icp = InfoCacheParam("zk", map) - val icps = icp :: Nil - - before { - InfoCacheInstance.initInstance(icps, name) - InfoCacheInstance.init - } - - test ("others") { - InfoCacheInstance.available should be (true) - - val keys = List[String]( - "key1", "key2" - ) - val info = Map[String, String]( - ("key1" -> "value1"), - ("key2" -> "value2") - ) - - InfoCacheInstance.cacheInfo(info) should be (true) - InfoCacheInstance.readInfo(keys) should be (info) - InfoCacheInstance.deleteInfo(keys) -// InfoCacheInstance.readInfo(keys) should be (Map[String, String]()) - - } - - after { - InfoCacheInstance.close() - } - -} +///* +//Licensed to the Apache Software Foundation (ASF) under one +//or more contributor license agreements. See the NOTICE file +//distributed with this work for additional information +//regarding copyright ownership. The ASF licenses this file +//to you under the Apache License, Version 2.0 (the +//"License"); you may not use this file except in compliance +//with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +//Unless required by applicable law or agreed to in writing, +//software distributed under the License is distributed on an +//"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +//KIND, either express or implied. See the License for the +//specific language governing permissions and limitations +//under the License. +//*/ +//package org.apache.griffin.measure.cache +// +//import java.util.Date +//import java.util.concurrent.{Executors, ThreadPoolExecutor, TimeUnit} +// +//import org.apache.curator.framework.recipes.locks.InterProcessMutex +//import org.apache.curator.framework.{CuratorFramework, CuratorFrameworkFactory} +//import org.apache.curator.retry.ExponentialBackoffRetry +//import org.apache.griffin.measure.cache.info.InfoCacheInstance +//import org.apache.griffin.measure.config.params.env.InfoCacheParam +//import org.junit.runner.RunWith +//import org.scalatest.junit.JUnitRunner +//import org.scalatest.{BeforeAndAfter, FunSuite, Matchers} +// +//import scala.util.{Failure, Try} +// +//@RunWith(classOf[JUnitRunner]) +//class InfoCacheInstanceTest extends FunSuite with Matchers with BeforeAndAfter { +// +// val map = Map[String, Any]( +// ("hosts" -> "localhost:2181"), +// ("namespace" -> "griffin/infocache"), +// ("lock.path" -> "lock"), +// ("mode" -> "persist"), +// ("init.clear" -> true), +// ("close.clear" -> false) +// ) +// val name = "ttt" +// +// val icp = InfoCacheParam("zk", map) +// val icps = icp :: Nil +// +// before { +// InfoCacheInstance.initInstance(icps, name) +// InfoCacheInstance.init +// } +// +// test ("others") { +// InfoCacheInstance.available should be (true) +// +// val keys = List[String]( +// "key1", "key2" +// ) +// val info = Map[String, String]( +// ("key1" -> "value1"), +// ("key2" -> "value2") +// ) +// +// InfoCacheInstance.cacheInfo(info) should be (true) +// InfoCacheInstance.readInfo(keys) should be (info) +// InfoCacheInstance.deleteInfo(keys) +//// InfoCacheInstance.readInfo(keys) should be (Map[String, String]()) +// +// } +// +// after { +// InfoCacheInstance.close() +// } +// +//} diff --git a/measure/src/test/scala/org/apache/griffin/measure/cache/ZKCacheLockTest.scala b/measure/src/test/scala/org/apache/griffin/measure/cache/ZKCacheLockTest.scala index 8fb400dc7..271529c34 100644 --- a/measure/src/test/scala/org/apache/griffin/measure/cache/ZKCacheLockTest.scala +++ b/measure/src/test/scala/org/apache/griffin/measure/cache/ZKCacheLockTest.scala @@ -1,84 +1,84 @@ -/* -Licensed to the Apache Software Foundation (ASF) under one -or more contributor license agreements. See the NOTICE file -distributed with this work for additional information -regarding copyright ownership. The ASF licenses this file -to you under the Apache License, Version 2.0 (the -"License"); you may not use this file except in compliance -with the License. You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, -software distributed under the License is distributed on an -"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -KIND, either express or implied. See the License for the -specific language governing permissions and limitations -under the License. -*/ -package org.apache.griffin.measure.cache - -import java.util.Date -import java.util.concurrent.{Executors, ThreadPoolExecutor, TimeUnit} - -import org.apache.curator.framework.recipes.locks.InterProcessMutex -import org.apache.curator.framework.{CuratorFramework, CuratorFrameworkFactory} -import org.apache.curator.retry.ExponentialBackoffRetry -import org.apache.griffin.measure.cache.info.ZKInfoCache -import org.junit.runner.RunWith -import org.scalatest.junit.JUnitRunner -import org.scalatest.{BeforeAndAfter, FunSuite, Matchers} - -import scala.util.{Failure, Try} - -@RunWith(classOf[JUnitRunner]) -class ZKCacheLockTest extends FunSuite with Matchers with BeforeAndAfter { - - val map = Map[String, Any]( - ("hosts" -> "localhost:2181"), - ("namespace" -> "griffin/infocache"), - ("lock.path" -> "lock"), - ("mode" -> "persist"), - ("init.clear" -> true), - ("close.clear" -> false) - ) - val name = "ttt" - - val ic = ZKInfoCache(map, name) - - before { - ic.init - } - - test ("lock") { - - case class Proc(n: Int) extends Runnable { - override def run(): Unit = { - val cl = ic.genLock("proc") - val b = cl.lock(2, TimeUnit.SECONDS) - try { - println(s"${n}: ${b}") - if (b) Thread.sleep(3000) - } finally { - cl.unlock() - } - } - } - - val pool = Executors.newFixedThreadPool(5).asInstanceOf[ThreadPoolExecutor] - val t = 0 until 10 - t.foreach(a => pool.submit(Proc(a))) - - pool.shutdown() - val t1 = new Date() - println(s"${t1}: pool shut down") - pool.awaitTermination(20, TimeUnit.SECONDS) - val t2 = new Date() - println(s"${t2}: pool shut down done [${t2.getTime - t1.getTime}]") - } - - after { - ic.close() - } - -} +///* +//Licensed to the Apache Software Foundation (ASF) under one +//or more contributor license agreements. See the NOTICE file +//distributed with this work for additional information +//regarding copyright ownership. The ASF licenses this file +//to you under the Apache License, Version 2.0 (the +//"License"); you may not use this file except in compliance +//with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +//Unless required by applicable law or agreed to in writing, +//software distributed under the License is distributed on an +//"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +//KIND, either express or implied. See the License for the +//specific language governing permissions and limitations +//under the License. +//*/ +//package org.apache.griffin.measure.cache +// +//import java.util.Date +//import java.util.concurrent.{Executors, ThreadPoolExecutor, TimeUnit} +// +//import org.apache.curator.framework.recipes.locks.InterProcessMutex +//import org.apache.curator.framework.{CuratorFramework, CuratorFrameworkFactory} +//import org.apache.curator.retry.ExponentialBackoffRetry +//import org.apache.griffin.measure.cache.info.ZKInfoCache +//import org.junit.runner.RunWith +//import org.scalatest.junit.JUnitRunner +//import org.scalatest.{BeforeAndAfter, FunSuite, Matchers} +// +//import scala.util.{Failure, Try} +// +//@RunWith(classOf[JUnitRunner]) +//class ZKCacheLockTest extends FunSuite with Matchers with BeforeAndAfter { +// +// val map = Map[String, Any]( +// ("hosts" -> "localhost:2181"), +// ("namespace" -> "griffin/infocache"), +// ("lock.path" -> "lock"), +// ("mode" -> "persist"), +// ("init.clear" -> true), +// ("close.clear" -> false) +// ) +// val name = "ttt" +// +// val ic = ZKInfoCache(map, name) +// +// before { +// ic.init +// } +// +// test ("lock") { +// +// case class Proc(n: Int) extends Runnable { +// override def run(): Unit = { +// val cl = ic.genLock("proc") +// val b = cl.lock(2, TimeUnit.SECONDS) +// try { +// println(s"${n}: ${b}") +// if (b) Thread.sleep(3000) +// } finally { +// cl.unlock() +// } +// } +// } +// +// val pool = Executors.newFixedThreadPool(5).asInstanceOf[ThreadPoolExecutor] +// val t = 0 until 10 +// t.foreach(a => pool.submit(Proc(a))) +// +// pool.shutdown() +// val t1 = new Date() +// println(s"${t1}: pool shut down") +// pool.awaitTermination(20, TimeUnit.SECONDS) +// val t2 = new Date() +// println(s"${t2}: pool shut down done [${t2.getTime - t1.getTime}]") +// } +// +// after { +// ic.close() +// } +// +//} diff --git a/measure/src/test/scala/org/apache/griffin/measure/cache/ZKInfoCacheTest.scala b/measure/src/test/scala/org/apache/griffin/measure/cache/ZKInfoCacheTest.scala index 8bd79952d..086170a58 100644 --- a/measure/src/test/scala/org/apache/griffin/measure/cache/ZKInfoCacheTest.scala +++ b/measure/src/test/scala/org/apache/griffin/measure/cache/ZKInfoCacheTest.scala @@ -1,90 +1,90 @@ -/* -Licensed to the Apache Software Foundation (ASF) under one -or more contributor license agreements. See the NOTICE file -distributed with this work for additional information -regarding copyright ownership. The ASF licenses this file -to you under the Apache License, Version 2.0 (the -"License"); you may not use this file except in compliance -with the License. You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, -software distributed under the License is distributed on an -"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -KIND, either express or implied. See the License for the -specific language governing permissions and limitations -under the License. -*/ -package org.apache.griffin.measure.cache - -import java.util.Date -import java.util.concurrent.{Executors, ThreadPoolExecutor, TimeUnit} - -import org.apache.curator.framework.recipes.locks.InterProcessMutex -import org.apache.curator.framework.{CuratorFramework, CuratorFrameworkFactory} -import org.apache.curator.retry.ExponentialBackoffRetry -import org.apache.griffin.measure.cache.info.ZKInfoCache -import org.junit.runner.RunWith -import org.scalatest.junit.JUnitRunner -import org.scalatest.{BeforeAndAfter, FunSuite, Matchers} - -import scala.util.{Failure, Try} - -@RunWith(classOf[JUnitRunner]) -class ZKInfoCacheTest extends FunSuite with Matchers with BeforeAndAfter { - - val map = Map[String, Any]( - ("hosts" -> "localhost:2181"), - ("namespace" -> "griffin/infocache"), - ("lock.path" -> "lock"), - ("mode" -> "persist"), - ("init.clear" -> true), - ("close.clear" -> false) - ) - val name = "ttt" - - test ("available") { - val ic = ZKInfoCache(map, name) - ic.init - - ic.available should be (true) - - ic.close - } - - test ("cacheInfo and readInfo") { - val ic = ZKInfoCache(map, name) - ic.init - - val keys = List[String]( - "key1", "key2" - ) - val info = Map[String, String]( - ("key1" -> "value1"), - ("key2" -> "value2") - ) - - ic.cacheInfo(info) should be (true) - ic.readInfo(keys) should be (info) - ic.deleteInfo(keys) - ic.readInfo(keys) should be (Map[String, String]()) - - ic.close - } - - test ("genLock") { - val ic = ZKInfoCache(map, name) - ic.init - - val lock1 = ic.genLock("ttt") - val lock2 = ic.genLock("ttt") - lock1.lock(5, TimeUnit.SECONDS) - lock2.lock(5, TimeUnit.SECONDS) - lock1.unlock - lock2.unlock - - ic.close - } - -} +///* +//Licensed to the Apache Software Foundation (ASF) under one +//or more contributor license agreements. See the NOTICE file +//distributed with this work for additional information +//regarding copyright ownership. The ASF licenses this file +//to you under the Apache License, Version 2.0 (the +//"License"); you may not use this file except in compliance +//with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +//Unless required by applicable law or agreed to in writing, +//software distributed under the License is distributed on an +//"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +//KIND, either express or implied. See the License for the +//specific language governing permissions and limitations +//under the License. +//*/ +//package org.apache.griffin.measure.cache +// +//import java.util.Date +//import java.util.concurrent.{Executors, ThreadPoolExecutor, TimeUnit} +// +//import org.apache.curator.framework.recipes.locks.InterProcessMutex +//import org.apache.curator.framework.{CuratorFramework, CuratorFrameworkFactory} +//import org.apache.curator.retry.ExponentialBackoffRetry +//import org.apache.griffin.measure.cache.info.ZKInfoCache +//import org.junit.runner.RunWith +//import org.scalatest.junit.JUnitRunner +//import org.scalatest.{BeforeAndAfter, FunSuite, Matchers} +// +//import scala.util.{Failure, Try} +// +//@RunWith(classOf[JUnitRunner]) +//class ZKInfoCacheTest extends FunSuite with Matchers with BeforeAndAfter { +// +// val map = Map[String, Any]( +// ("hosts" -> "localhost:2181"), +// ("namespace" -> "griffin/infocache"), +// ("lock.path" -> "lock"), +// ("mode" -> "persist"), +// ("init.clear" -> true), +// ("close.clear" -> false) +// ) +// val name = "ttt" +// +// test ("available") { +// val ic = ZKInfoCache(map, name) +// ic.init +// +// ic.available should be (true) +// +// ic.close +// } +// +// test ("cacheInfo and readInfo") { +// val ic = ZKInfoCache(map, name) +// ic.init +// +// val keys = List[String]( +// "key1", "key2" +// ) +// val info = Map[String, String]( +// ("key1" -> "value1"), +// ("key2" -> "value2") +// ) +// +// ic.cacheInfo(info) should be (true) +// ic.readInfo(keys) should be (info) +// ic.deleteInfo(keys) +// ic.readInfo(keys) should be (Map[String, String]()) +// +// ic.close +// } +// +// test ("genLock") { +// val ic = ZKInfoCache(map, name) +// ic.init +// +// val lock1 = ic.genLock("ttt") +// val lock2 = ic.genLock("ttt") +// lock1.lock(5, TimeUnit.SECONDS) +// lock2.lock(5, TimeUnit.SECONDS) +// lock1.unlock +// lock2.unlock +// +// ic.close +// } +// +//} diff --git a/measure/src/test/scala/org/apache/griffin/measure/rule/RuleAnalyzerTest.scala b/measure/src/test/scala/org/apache/griffin/measure/rule/RuleAnalyzerTest.scala index 5b1d1eee6..47b36e805 100644 --- a/measure/src/test/scala/org/apache/griffin/measure/rule/RuleAnalyzerTest.scala +++ b/measure/src/test/scala/org/apache/griffin/measure/rule/RuleAnalyzerTest.scala @@ -27,7 +27,7 @@ import org.scalatest.{BeforeAndAfter, FunSuite, Matchers} class RuleAnalyzerTest extends FunSuite with BeforeAndAfter with Matchers { test ("rule analyze") { - val rule = "$source.name = $target.name AND $source.age = $target.age + (2 * 5) WHEN $source.born > (6 - 2 * 2)" + val rule = "$source.name = $target.name AND $source.age = $target.age + (2 * 5) AND $source.born > (6 - 2 * 2)" val evaluateRuleParam = EvaluateRuleParam(1.0, rule) val ruleFactory = RuleFactory(evaluateRuleParam) val statement = ruleFactory.generateRule diff --git a/measure/src/test/scala/org/apache/griffin/measure/rule/RuleParserTest.scala b/measure/src/test/scala/org/apache/griffin/measure/rule/RuleParserTest.scala index ef57fb1d8..afcb95d25 100644 --- a/measure/src/test/scala/org/apache/griffin/measure/rule/RuleParserTest.scala +++ b/measure/src/test/scala/org/apache/griffin/measure/rule/RuleParserTest.scala @@ -194,9 +194,9 @@ class RuleParserTest extends FunSuite with Matchers with BeforeAndAfter { result1.successful should be (true) result1.get.desc should be ("$source['name'] = $target['name'] AND $source['age'] = $target['age']") - val rule2 = "$source.name = $target.name AND $source.age = $target.age WHEN $source.id > 1000" + val rule2 = "$source.name = $target.name AND $source.age = $target.age AND $source.id > 1000" val result2 = ruleParser.parseAll(ruleParser.rule, rule2) result2.successful should be (true) - result2.get.desc should be ("$source['name'] = $target['name'] AND $source['age'] = $target['age'] when $source['id'] > 1000") + result2.get.desc should be ("$source['name'] = $target['name'] AND $source['age'] = $target['age'] AND $source['id'] > 1000") } } From 822f6989d0e98d8995988228a68b8fcec3a383cd Mon Sep 17 00:00:00 2001 From: Lionel Liu Date: Sun, 23 Jul 2017 17:14:18 +0800 Subject: [PATCH 030/111] t1 --- .../src/main/resources/config-streaming.json | 2 ++ measure/src/main/resources/config.json | 2 ++ .../apache/griffin/measure/Application.scala | 14 ++++++---- .../griffin/measure/algo/MeasureType.scala | 2 +- .../griffin/measure/algo/ProcessType.scala | 26 +++++++++++++++++++ .../config/params/user/UserParam.scala | 1 + .../src/test/resources/config-profile.json | 2 ++ .../src/test/resources/config-streaming.json | 2 ++ .../src/test/resources/config-streaming1.json | 2 ++ measure/src/test/resources/config.json | 2 ++ measure/src/test/resources/config1.json | 2 ++ 11 files changed, 51 insertions(+), 6 deletions(-) create mode 100644 measure/src/main/scala/org/apache/griffin/measure/algo/ProcessType.scala diff --git a/measure/src/main/resources/config-streaming.json b/measure/src/main/resources/config-streaming.json index 023d63b9d..170517471 100644 --- a/measure/src/main/resources/config-streaming.json +++ b/measure/src/main/resources/config-streaming.json @@ -2,6 +2,8 @@ "name": "accu2", "type": "accuracy", + "process.type": "steaming", + "source": { "type": "kafka", "version": "0.8", diff --git a/measure/src/main/resources/config.json b/measure/src/main/resources/config.json index edd2e6a49..ab32b75cf 100644 --- a/measure/src/main/resources/config.json +++ b/measure/src/main/resources/config.json @@ -2,6 +2,8 @@ "name": "accu1", "type": "accuracy", + "process.type": "batch", + "source": { "type": "hive", "version": "1.2", diff --git a/measure/src/main/scala/org/apache/griffin/measure/Application.scala b/measure/src/main/scala/org/apache/griffin/measure/Application.scala index 6d47f3134..3d95b0e43 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/Application.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/Application.scala @@ -19,7 +19,8 @@ under the License. package org.apache.griffin.measure import org.apache.griffin.measure.algo._ -import org.apache.griffin.measure.algo.batch.{BatchAccuracyAlgo, BatchProfileAlgo} +import org.apache.griffin.measure.algo.batch._ +import org.apache.griffin.measure.algo.streaming._ import org.apache.griffin.measure.config.params._ import org.apache.griffin.measure.config.params.env._ import org.apache.griffin.measure.config.params.user._ @@ -80,11 +81,14 @@ object Application extends Loggable { // choose algorithm val dqType = allParam.userParam.dqType - val algo: Algo = dqType match { - case MeasureType.accuracy() => BatchAccuracyAlgo(allParam) - case MeasureType.profile() => BatchProfileAlgo(allParam) + val procType = allParam.userParam.procType + val algo: Algo = (dqType, procType) match { + case (MeasureType.accuracy(), ProcessType.batch()) => BatchAccuracyAlgo(allParam) + case (MeasureType.profile(), ProcessType.batch()) => BatchProfileAlgo(allParam) + case (MeasureType.accuracy(), ProcessType.streaming()) => StreamingAccuracyAlgo(allParam) +// case (MeasureType.profile(), ProcessType.streaming()) => StreamingProfileAlgo(allParam) case _ => { - error(s"${dqType} is unsupported dq type!") + error(s"${dqType} with ${procType} is unsupported dq type!") sys.exit(-4) } } diff --git a/measure/src/main/scala/org/apache/griffin/measure/algo/MeasureType.scala b/measure/src/main/scala/org/apache/griffin/measure/algo/MeasureType.scala index 6bfff983c..23d4dac43 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/algo/MeasureType.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/algo/MeasureType.scala @@ -21,6 +21,6 @@ package org.apache.griffin.measure.algo object MeasureType { val accuracy = """^(?i)accuracy$""".r - val profile = """^(?i)profile""".r + val profile = """^(?i)profile$""".r } diff --git a/measure/src/main/scala/org/apache/griffin/measure/algo/ProcessType.scala b/measure/src/main/scala/org/apache/griffin/measure/algo/ProcessType.scala new file mode 100644 index 000000000..5a85c7c00 --- /dev/null +++ b/measure/src/main/scala/org/apache/griffin/measure/algo/ProcessType.scala @@ -0,0 +1,26 @@ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ +package org.apache.griffin.measure.algo + +object ProcessType { + + val batch = """^(?i)batch$""".r + val streaming = """^(?i)streaming$""".r + +} diff --git a/measure/src/main/scala/org/apache/griffin/measure/config/params/user/UserParam.scala b/measure/src/main/scala/org/apache/griffin/measure/config/params/user/UserParam.scala index cd44e0b15..df0647cc5 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/config/params/user/UserParam.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/config/params/user/UserParam.scala @@ -25,6 +25,7 @@ import org.apache.griffin.measure.config.params.Param @JsonInclude(Include.NON_NULL) case class UserParam(@JsonProperty("name") name: String, @JsonProperty("type") dqType: String, + @JsonProperty("process.type") procType: String, @JsonProperty("source") sourceParam: DataConnectorParam, @JsonProperty("target") targetParam: DataConnectorParam, @JsonProperty("evaluateRule") evaluateRuleParam: EvaluateRuleParam diff --git a/measure/src/test/resources/config-profile.json b/measure/src/test/resources/config-profile.json index 6b82d7fb9..2529def39 100644 --- a/measure/src/test/resources/config-profile.json +++ b/measure/src/test/resources/config-profile.json @@ -2,6 +2,8 @@ "name": "prof1", "type": "profile", + "process.type": "batch", + "source": { "type": "avro", "version": "1.7", diff --git a/measure/src/test/resources/config-streaming.json b/measure/src/test/resources/config-streaming.json index 8a26887af..7262b245f 100644 --- a/measure/src/test/resources/config-streaming.json +++ b/measure/src/test/resources/config-streaming.json @@ -2,6 +2,8 @@ "name": "accu2", "type": "accuracy", + "process.type": "streaming", + "source": { "type": "kafka", "version": "0.8", diff --git a/measure/src/test/resources/config-streaming1.json b/measure/src/test/resources/config-streaming1.json index d827f0f7b..546520185 100644 --- a/measure/src/test/resources/config-streaming1.json +++ b/measure/src/test/resources/config-streaming1.json @@ -2,6 +2,8 @@ "name": "accu2", "type": "accuracy", + "process.type": "streaming", + "source": { "type": "kafka", "version": "0.8", diff --git a/measure/src/test/resources/config.json b/measure/src/test/resources/config.json index 4e3e96526..08a60212e 100644 --- a/measure/src/test/resources/config.json +++ b/measure/src/test/resources/config.json @@ -2,6 +2,8 @@ "name": "accu1", "type": "accuracy", + "process.type": "batch", + "source": { "type": "avro", "version": "1.7", diff --git a/measure/src/test/resources/config1.json b/measure/src/test/resources/config1.json index d7290ba4e..16c265d93 100644 --- a/measure/src/test/resources/config1.json +++ b/measure/src/test/resources/config1.json @@ -2,6 +2,8 @@ "name": "accu-test", "type": "accuracy", + "process.type": "batch", + "source": { "type": "hive", "version": "1.2", From b187c62dfa3ffbfa6a32047951b1777e33fabd6a Mon Sep 17 00:00:00 2001 From: Lionel Liu Date: Mon, 24 Jul 2017 16:23:07 +0800 Subject: [PATCH 031/111] v2 --- .../streaming/StreamingAccuracyProcess.scala | 7 ++++-- .../measure/cache/info/ZKInfoCache.scala | 23 ++++++++++++++++--- .../griffin/measure/persist/HdfsPersist.scala | 2 +- .../src/test/resources/config-streaming.json | 2 +- 4 files changed, 27 insertions(+), 7 deletions(-) diff --git a/measure/src/main/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyProcess.scala b/measure/src/main/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyProcess.scala index e3fabc1cf..a5ec7d1e4 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyProcess.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyProcess.scala @@ -47,6 +47,7 @@ case class StreamingAccuracyProcess(sourceDataConnector: DirectDataConnector, def run(): Unit = { // println(s"cache count: ${cacheResultProcesser.cacheGroup.size}") val updateTime = new Date().getTime + println(s"===== [${updateTime}] process begins =====") val locked = lock.lock(5, TimeUnit.SECONDS) if (locked) { try { @@ -67,8 +68,8 @@ case class StreamingAccuracyProcess(sourceDataConnector: DirectDataConnector, sourceData.cache targetData.cache -// println(s"sourceData.count: ${sourceData.count}") -// println(s"targetData.count: ${targetData.count}") + println(s"sourceData.count: ${sourceData.count}") + println(s"targetData.count: ${targetData.count}") // accuracy algorithm val (accuResult, missingRdd, matchedRdd) = accuracy(sourceData, targetData, ruleAnalyzer) @@ -171,6 +172,8 @@ case class StreamingAccuracyProcess(sourceDataConnector: DirectDataConnector, lock.unlock() } } + val endTime = new Date().getTime + println(s"===== [${updateTime}] process ends, using ${endTime - updateTime} ms =====") } // clean old data and old result cache diff --git a/measure/src/main/scala/org/apache/griffin/measure/cache/info/ZKInfoCache.scala b/measure/src/main/scala/org/apache/griffin/measure/cache/info/ZKInfoCache.scala index 479243806..8b62fa425 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/cache/info/ZKInfoCache.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/cache/info/ZKInfoCache.scala @@ -22,6 +22,7 @@ import org.apache.curator.framework.imps.CuratorFrameworkState import org.apache.curator.framework.recipes.locks.InterProcessMutex import org.apache.curator.framework.{CuratorFramework, CuratorFrameworkFactory} import org.apache.curator.retry.ExponentialBackoffRetry +import org.apache.curator.utils.ZKPaths import org.apache.griffin.measure.cache.lock.ZKCacheLock import org.apache.zookeeper.CreateMode @@ -39,7 +40,7 @@ case class ZKInfoCache(config: Map[String, Any], metricName: String) extends Inf val PersistRegex = """^(?i)persist$""".r val EphemeralRegex = """^(?i)ephemeral$""".r - final val separator = "/" + final val separator = ZKPaths.PATH_SEPARATOR val hosts = config.getOrElse(Hosts, "").toString val namespace = config.getOrElse(Namespace, "").toString @@ -70,7 +71,10 @@ case class ZKInfoCache(config: Map[String, Any], metricName: String) extends Inf def init(): Unit = { client.start() + info("start zk info cache") client.usingNamespace(cacheNamespace) + info(s"init with namespace: ${cacheNamespace}") + deleteInfo(lockPath :: Nil) if (initClear) { clearInfo } @@ -87,6 +91,7 @@ case class ZKInfoCache(config: Map[String, Any], metricName: String) extends Inf if (closeClear) { clearInfo } + info("close zk info cache") client.close() } @@ -111,11 +116,12 @@ case class ZKInfoCache(config: Map[String, Any], metricName: String) extends Inf } def clearInfo(): Unit = { - delete("/") +// delete("/") + info("clear info") } def listKeys(p: String): List[String] = { - client.getChildren().forPath(path(p)).asScala.toList + children(path(p)) } def genLock(s: String): ZKCacheLock = { @@ -127,6 +133,17 @@ case class ZKInfoCache(config: Map[String, Any], metricName: String) extends Inf if (k.startsWith(separator)) k else separator + k } + private def children(path: String): List[String] = { + try { + client.getChildren().forPath(path).asScala.toList + } catch { + case e: Throwable => { + error(s"list ${path} error: ${e.getMessage}") + Nil + } + } + } + private def createOrUpdate(path: String, content: String): Boolean = { if (checkExists(path)) { update(path, content) diff --git a/measure/src/main/scala/org/apache/griffin/measure/persist/HdfsPersist.scala b/measure/src/main/scala/org/apache/griffin/measure/persist/HdfsPersist.scala index cbaa4c2bc..97786c47a 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/persist/HdfsPersist.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/persist/HdfsPersist.scala @@ -191,7 +191,7 @@ case class HdfsPersist(config: Map[String, Any], metricName: String, timeStamp: private def persistRecords(hdfsPath: String, records: Iterable[String]): Unit = { val recStr = records.mkString("\n") - HdfsUtil.appendContent(hdfsPath, recStr) + HdfsUtil.writeContent(hdfsPath, recStr) } def log(rt: Long, msg: String): Unit = { diff --git a/measure/src/test/resources/config-streaming.json b/measure/src/test/resources/config-streaming.json index 7262b245f..697fb7b6d 100644 --- a/measure/src/test/resources/config-streaming.json +++ b/measure/src/test/resources/config-streaming.json @@ -1,5 +1,5 @@ { - "name": "accu2", + "name": "accu1", "type": "accuracy", "process.type": "streaming", From 0c3a5e287c521b66b5985c15a1679e6935a4d54c Mon Sep 17 00:00:00 2001 From: Lionel Liu Date: Thu, 27 Jul 2017 15:23:14 +0800 Subject: [PATCH 032/111] v1 --- .../apache/griffin/measure/Application.scala | 8 + .../griffin/measure/persist/HttpPersist.scala | 11 +- .../measure/persist/PersistThreadPool.scala | 62 +++ .../griffin/measure/utils/HdfsUtil.scala | 2 +- .../griffin/measure/utils/HttpUtil.scala | 18 +- .../streaming/StreamingAccuracyAlgoTest.scala | 503 +++++++++--------- 6 files changed, 351 insertions(+), 253 deletions(-) create mode 100644 measure/src/main/scala/org/apache/griffin/measure/persist/PersistThreadPool.scala diff --git a/measure/src/main/scala/org/apache/griffin/measure/Application.scala b/measure/src/main/scala/org/apache/griffin/measure/Application.scala index 3d95b0e43..7f385f4cf 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/Application.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/Application.scala @@ -27,6 +27,7 @@ import org.apache.griffin.measure.config.params.user._ import org.apache.griffin.measure.config.reader._ import org.apache.griffin.measure.config.validator.AllParamValidator import org.apache.griffin.measure.log.Loggable +import org.apache.griffin.measure.persist.PersistThreadPool import scala.util.{Failure, Success, Try} @@ -103,6 +104,9 @@ object Application extends Loggable { info("calculation finished") } } + + // shut down + shutdown } private def readParamFile[T <: Param](file: String, fsType: String)(implicit m : Manifest[T]): Try[T] = { @@ -115,4 +119,8 @@ object Application extends Loggable { allParamValidator.validate(allParam) } + private def shutdown(): Unit = { + PersistThreadPool.shutdown + } + } diff --git a/measure/src/main/scala/org/apache/griffin/measure/persist/HttpPersist.scala b/measure/src/main/scala/org/apache/griffin/measure/persist/HttpPersist.scala index ef90912d3..6d5bac3e5 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/persist/HttpPersist.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/persist/HttpPersist.scala @@ -62,8 +62,15 @@ case class HttpPersist(config: Map[String, Any], metricName: String, timeStamp: // post val params = Map[String, Object]() val header = Map[String, Object]() - val status = HttpUtil.httpRequest(api, method, params, header, data) - info(s"${method} to ${api} response status: ${status}") + + def func(): Boolean = { + HttpUtil.httpRequest(api, method, params, header, data) + } + + PersistThreadPool.addTask(func _, 10) + +// val status = HttpUtil.httpRequest(api, method, params, header, data) +// info(s"${method} to ${api} response status: ${status}") } catch { case e: Throwable => error(e.getMessage) } diff --git a/measure/src/main/scala/org/apache/griffin/measure/persist/PersistThreadPool.scala b/measure/src/main/scala/org/apache/griffin/measure/persist/PersistThreadPool.scala new file mode 100644 index 000000000..7993aab21 --- /dev/null +++ b/measure/src/main/scala/org/apache/griffin/measure/persist/PersistThreadPool.scala @@ -0,0 +1,62 @@ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ +package org.apache.griffin.measure.persist + +import java.util.concurrent.{Executors, ThreadPoolExecutor, TimeUnit} + +object PersistThreadPool { + + private val pool: ThreadPoolExecutor = Executors.newFixedThreadPool(10).asInstanceOf[ThreadPoolExecutor] + val MAX_RETRY = 100 + + def shutdown(): Unit = { + pool.shutdown() + pool.awaitTermination(10, TimeUnit.SECONDS) + } + + def addTask(func: () => Boolean, retry: Int): Unit = { + val r = if (retry < 0) MAX_RETRY else retry + println(s"add task, current task num: ${pool.getQueue.size}") + pool.submit(Task(func, r)) + } + + case class Task(func: () => Boolean, retry: Int) extends Runnable { + + override def run(): Unit = { + try { + var i = retry + var suc = false + while (!suc && i > 0) { + if (func()) { + println("task success") + suc = true + } else i = i - 1 + } + if (!suc) fail(s"retried for ${retry} times") + } catch { + case e: Throwable => fail(s"${e.getMessage}") + } + } + + def fail(msg: String): Unit = { + println(s"task fails: ${msg}") + } + } + +} diff --git a/measure/src/main/scala/org/apache/griffin/measure/utils/HdfsUtil.scala b/measure/src/main/scala/org/apache/griffin/measure/utils/HdfsUtil.scala index 3172420b8..c97a2112b 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/utils/HdfsUtil.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/utils/HdfsUtil.scala @@ -27,7 +27,7 @@ object HdfsUtil { private val conf = new Configuration() conf.set("dfs.support.append", "true") -// conf.set("fs.defaultFS", "hdfs://localhost") // debug @localhost + conf.set("fs.defaultFS", "hdfs://localhost") // debug @localhost private val dfs = FileSystem.get(conf) diff --git a/measure/src/main/scala/org/apache/griffin/measure/utils/HttpUtil.scala b/measure/src/main/scala/org/apache/griffin/measure/utils/HttpUtil.scala index d5a3d3f3a..e016b606a 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/utils/HttpUtil.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/utils/HttpUtil.scala @@ -27,17 +27,23 @@ object HttpUtil { val PUT_REGEX = """^(?i)put$""".r val DELETE_REGEX = """^(?i)delete$""".r - def postData(url: String, params: Map[String, Object], headers: Map[String, Object], data: String): String = { + def postData(url: String, params: Map[String, Object], headers: Map[String, Object], data: String): Boolean = { val response = Http(url).params(convertObjMap2StrMap(params)).headers(convertObjMap2StrMap(headers)).postData(data).asString - response.code.toString + response.isSuccess } - def httpRequest(url: String, method: String, params: Map[String, Object], headers: Map[String, Object], data: String): String = { + def httpRequest(url: String, method: String, params: Map[String, Object], headers: Map[String, Object], data: String): Boolean = { val httpReq = Http(url).params(convertObjMap2StrMap(params)).headers(convertObjMap2StrMap(headers)) method match { - case POST_REGEX() => httpReq.postData(data).asString.code.toString - case PUT_REGEX() => httpReq.put(data).asString.code.toString - case _ => "wrong method" + case POST_REGEX() => { + val res = httpReq.postData(data).asString + res.isSuccess + } + case PUT_REGEX() => { + val res = httpReq.put(data).asString + res.isSuccess + } + case _ => false } } diff --git a/measure/src/test/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgoTest.scala b/measure/src/test/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgoTest.scala index 376eddad0..d92da30c1 100644 --- a/measure/src/test/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgoTest.scala +++ b/measure/src/test/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgoTest.scala @@ -1,250 +1,265 @@ -///* -//Licensed to the Apache Software Foundation (ASF) under one -//or more contributor license agreements. See the NOTICE file -//distributed with this work for additional information -//regarding copyright ownership. The ASF licenses this file -//to you under the Apache License, Version 2.0 (the -//"License"); you may not use this file except in compliance -//with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -//Unless required by applicable law or agreed to in writing, -//software distributed under the License is distributed on an -//"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -//KIND, either express or implied. See the License for the -//specific language governing permissions and limitations -//under the License. -//*/ -//package org.apache.griffin.measure.algo.streaming -// -//import java.util.Date -//import java.util.concurrent.TimeUnit -// -//import org.apache.griffin.measure.algo.batch.BatchAccuracyAlgo -//import org.apache.griffin.measure.cache.info.{InfoCacheInstance, TimeInfoCache} -//import org.apache.griffin.measure.cache.result._ -//import org.apache.griffin.measure.config.params._ -//import org.apache.griffin.measure.config.params.env._ -//import org.apache.griffin.measure.config.params.user._ -//import org.apache.griffin.measure.config.reader._ -//import org.apache.griffin.measure.config.validator._ -//import org.apache.griffin.measure.connector.direct.DirectDataConnector -//import org.apache.griffin.measure.connector.{DataConnector, DataConnectorFactory} -//import org.apache.griffin.measure.log.Loggable -//import org.apache.griffin.measure.persist.{Persist, PersistFactory, PersistType} -//import org.apache.griffin.measure.result._ -//import org.apache.griffin.measure.rule.expr._ -//import org.apache.griffin.measure.rule.{ExprValueUtil, RuleAnalyzer, RuleFactory} -//import org.apache.griffin.measure.utils.{HdfsUtil, TimeUtil} -//import org.apache.spark.rdd.RDD -//import org.apache.spark.sql.SQLContext -//import org.apache.spark.sql.hive.HiveContext -//import org.apache.spark.streaming.{Milliseconds, StreamingContext} -//import org.apache.spark.{SparkConf, SparkContext} -//import org.junit.runner.RunWith -//import org.scalatest.junit.JUnitRunner -//import org.scalatest.{BeforeAndAfter, FunSuite, Matchers} -// -//import scala.util.{Failure, Success, Try} -// -// -//@RunWith(classOf[JUnitRunner]) -//class StreamingAccuracyAlgoTest extends FunSuite with Matchers with BeforeAndAfter with Loggable { -// -// val envFile = "src/test/resources/env-streaming.json" -// val confFile = "src/test/resources/config-streaming.json" -// val envFsType = "local" -// val userFsType = "local" -// -// val args = Array(envFile, confFile) -// -// var sc: SparkContext = _ -// var sqlContext: SQLContext = _ -//// val ssc: StreamingContext = _ -// -// var allParam: AllParam = _ -// -// before { -// // read param files -// val envParam = readParamFile[EnvParam](envFile, envFsType) match { -// case Success(p) => p -// case Failure(ex) => { -// error(ex.getMessage) -// sys.exit(-2) -// } -// } -// val userParam = readParamFile[UserParam](confFile, userFsType) match { -// case Success(p) => p -// case Failure(ex) => { -// error(ex.getMessage) -// sys.exit(-2) -// } -// } -// allParam = AllParam(envParam, userParam) -// -// // validate param files -// validateParams(allParam) match { -// case Failure(ex) => { -// error(ex.getMessage) -// sys.exit(-3) -// } -// case _ => { -// info("params validation pass") -// } -// } -// -// val metricName = userParam.name -// val conf = new SparkConf().setMaster("local[*]").setAppName(metricName) -// sc = new SparkContext(conf) -// sc.setLogLevel(envParam.sparkParam.logLevel) -//// sqlContext = new SQLContext(sc) -// sqlContext = new HiveContext(sc) -// -//// val a = sqlContext.sql("select * from s1 limit 10") -//// // val a = sqlContext.sql("show tables") -//// a.show(10) -//// -//// val b = HdfsUtil.existPath("/griffin/streaming") -//// println(b) -// } -// -// test("algorithm") { -// val envParam = allParam.envParam -// val userParam = allParam.userParam -// val metricName = userParam.name -// val sparkParam = envParam.sparkParam -// val cleanerParam = envParam.cleanerParam -// -// val batchInterval = TimeUtil.milliseconds(sparkParam.batchInterval) match { -// case Some(interval) => Milliseconds(interval) -// case _ => throw new Exception("invalid batch interval") -// } -// val ssc = new StreamingContext(sc, batchInterval) -// ssc.checkpoint(sparkParam.cpDir) -// -//// def createContext(): StreamingContext = { -//// ; -//// } -// -// // start time -// val startTime = new Date().getTime() -// -// val persistFactory = PersistFactory(envParam.persistParams, metricName) -// -// // get persists to persist measure result -// val appPersist: Persist = persistFactory.getPersists(startTime) -// -// // get spark application id -// val applicationId = sc.applicationId -// -// // persist start id -// appPersist.start(applicationId) -// -// InfoCacheInstance.initInstance(envParam.infoCacheParams, metricName) -// InfoCacheInstance.init -// -// // generate rule from rule param, generate rule analyzer -// val ruleFactory = RuleFactory(userParam.evaluateRuleParam) -// val rule: StatementExpr = ruleFactory.generateRule() -// val ruleAnalyzer: RuleAnalyzer = RuleAnalyzer(rule) -// -// // const expr value map -// val constExprValueMap = ExprValueUtil.genExprValueMaps(None, ruleAnalyzer.constCacheExprs, Map[String, Any]()) -// val finalConstExprValueMap = ExprValueUtil.updateExprValueMaps(ruleAnalyzer.constFinalCacheExprs, constExprValueMap) -// val finalConstMap = finalConstExprValueMap.headOption match { -// case Some(m) => m -// case _ => Map[String, Any]() -// } -// -// // data connector -// val sourceDataConnector: DirectDataConnector = -// DataConnectorFactory.getDirectDataConnector(sqlContext, ssc, userParam.sourceParam, -// ruleAnalyzer.sourceRuleExprs, finalConstMap -// ) match { -// case Success(cntr) => { -// if (cntr.available) cntr -// else throw new Exception("source data connection error!") +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ +package org.apache.griffin.measure.algo.streaming + +import java.util.Date +import java.util.concurrent.TimeUnit + +import org.apache.griffin.measure.algo.batch.BatchAccuracyAlgo +import org.apache.griffin.measure.cache.info.{InfoCacheInstance, TimeInfoCache} +import org.apache.griffin.measure.cache.result._ +import org.apache.griffin.measure.config.params._ +import org.apache.griffin.measure.config.params.env._ +import org.apache.griffin.measure.config.params.user._ +import org.apache.griffin.measure.config.reader._ +import org.apache.griffin.measure.config.validator._ +import org.apache.griffin.measure.connector.direct.DirectDataConnector +import org.apache.griffin.measure.connector.{DataConnector, DataConnectorFactory} +import org.apache.griffin.measure.log.Loggable +import org.apache.griffin.measure.persist.{Persist, PersistFactory, PersistType} +import org.apache.griffin.measure.result._ +import org.apache.griffin.measure.rule.expr._ +import org.apache.griffin.measure.rule.{ExprValueUtil, RuleAnalyzer, RuleFactory} +import org.apache.griffin.measure.utils.{HdfsUtil, TimeUtil} +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.hive.HiveContext +import org.apache.spark.streaming.{Milliseconds, StreamingContext} +import org.apache.spark.{SparkConf, SparkContext} +import org.junit.runner.RunWith +import org.scalatest.junit.JUnitRunner +import org.scalatest.{BeforeAndAfter, FunSuite, Matchers} + +import scala.util.{Failure, Success, Try} + + +@RunWith(classOf[JUnitRunner]) +class StreamingAccuracyAlgoTest extends FunSuite with Matchers with BeforeAndAfter with Loggable { + + val envFile = "src/test/resources/env-streaming.json" + val confFile = "src/test/resources/config-streaming.json" + val envFsType = "local" + val userFsType = "local" + + val args = Array(envFile, confFile) + + var sc: SparkContext = _ + var sqlContext: SQLContext = _ +// val ssc: StreamingContext = _ + + var allParam: AllParam = _ + + before { + // read param files + val envParam = readParamFile[EnvParam](envFile, envFsType) match { + case Success(p) => p + case Failure(ex) => { + error(ex.getMessage) + sys.exit(-2) + } + } + val userParam = readParamFile[UserParam](confFile, userFsType) match { + case Success(p) => p + case Failure(ex) => { + error(ex.getMessage) + sys.exit(-2) + } + } + allParam = AllParam(envParam, userParam) + + // validate param files + validateParams(allParam) match { + case Failure(ex) => { + error(ex.getMessage) + sys.exit(-3) + } + case _ => { + info("params validation pass") + } + } + + val metricName = userParam.name + val conf = new SparkConf().setMaster("local[*]").setAppName(metricName) + sc = new SparkContext(conf) + sc.setLogLevel(envParam.sparkParam.logLevel) +// sqlContext = new SQLContext(sc) + sqlContext = new HiveContext(sc) + +// val a = sqlContext.sql("select * from s1 limit 10") +// // val a = sqlContext.sql("show tables") +// a.show(10) +// +// val b = HdfsUtil.existPath("/griffin/streaming") +// println(b) + } + + test("algorithm") { + val envParam = allParam.envParam + val userParam = allParam.userParam + val metricName = userParam.name + val sparkParam = envParam.sparkParam + val cleanerParam = envParam.cleanerParam + +// val ssc = StreamingContext.getOrCreate(sparkParam.cpDir, +// ( ) => { +// try { +// val batchInterval = TimeUtil.milliseconds(sparkParam.batchInterval) match { +// case Some(interval) => Milliseconds(interval) +// case _ => throw new Exception("invalid batch interval") +// } +// val ssc = new StreamingContext(sc, batchInterval) +// ssc.checkpoint(sparkParam.cpDir) +// ssc +// } catch { +// case runtime: RuntimeException => { +// throw runtime +// } // } -// case Failure(ex) => throw ex -// } -// val targetDataConnector: DirectDataConnector = -// DataConnectorFactory.getDirectDataConnector(sqlContext, ssc, userParam.targetParam, -// ruleAnalyzer.targetRuleExprs, finalConstMap -// ) match { -// case Success(cntr) => { -// if (cntr.available) cntr -// else throw new Exception("target data connection error!") +// }) + + val batchInterval = TimeUtil.milliseconds(sparkParam.batchInterval) match { + case Some(interval) => Milliseconds(interval) + case _ => throw new Exception("invalid batch interval") + } + val ssc = new StreamingContext(sc, batchInterval) + ssc.checkpoint(sparkParam.cpDir) + + // start time + val startTime = new Date().getTime() + + val persistFactory = PersistFactory(envParam.persistParams, metricName) + + // get persists to persist measure result + val appPersist: Persist = persistFactory.getPersists(startTime) + + // get spark application id + val applicationId = sc.applicationId + + // persist start id + appPersist.start(applicationId) + + InfoCacheInstance.initInstance(envParam.infoCacheParams, metricName) + InfoCacheInstance.init + + // generate rule from rule param, generate rule analyzer + val ruleFactory = RuleFactory(userParam.evaluateRuleParam) + val rule: StatementExpr = ruleFactory.generateRule() + val ruleAnalyzer: RuleAnalyzer = RuleAnalyzer(rule) + + // const expr value map + val constExprValueMap = ExprValueUtil.genExprValueMaps(None, ruleAnalyzer.constCacheExprs, Map[String, Any]()) + val finalConstExprValueMap = ExprValueUtil.updateExprValueMaps(ruleAnalyzer.constFinalCacheExprs, constExprValueMap) + val finalConstMap = finalConstExprValueMap.headOption match { + case Some(m) => m + case _ => Map[String, Any]() + } + + // data connector + val sourceDataConnector: DirectDataConnector = + DataConnectorFactory.getDirectDataConnector(sqlContext, ssc, userParam.sourceParam, + ruleAnalyzer.sourceRuleExprs, finalConstMap + ) match { + case Success(cntr) => { + if (cntr.available) cntr + else throw new Exception("source data connection error!") + } + case Failure(ex) => throw ex + } + val targetDataConnector: DirectDataConnector = + DataConnectorFactory.getDirectDataConnector(sqlContext, ssc, userParam.targetParam, + ruleAnalyzer.targetRuleExprs, finalConstMap + ) match { + case Success(cntr) => { + if (cntr.available) cntr + else throw new Exception("target data connection error!") + } + case Failure(ex) => throw ex + } + + val cacheResultProcesser = CacheResultProcesser() + + // init data stream + sourceDataConnector.init() + targetDataConnector.init() + + // my algo + val algo = StreamingAccuracyAlgo(allParam) + + val streamingAccuracyProcess = StreamingAccuracyProcess( + sourceDataConnector, targetDataConnector, + ruleAnalyzer, cacheResultProcesser, persistFactory, appPersist) + + val processInterval = TimeUtil.milliseconds(sparkParam.processInterval) match { + case Some(interval) => interval + case _ => throw new Exception("invalid batch interval") + } + val process = TimingProcess(processInterval, streamingAccuracyProcess) + + // clean thread +// case class Clean() extends Runnable { +// val lock = InfoCacheInstance.genLock("clean") +// def run(): Unit = { +// val locked = lock.lock(5, TimeUnit.SECONDS) +// if (locked) { +// try { +// sourceDataConnector.cleanData +// targetDataConnector.cleanData +// } finally { +// lock.unlock() +// } // } -// case Failure(ex) => throw ex // } -// -// val cacheResultProcesser = CacheResultProcesser() -// -// // init data stream -// sourceDataConnector.init() -// targetDataConnector.init() -// -// // my algo -// val algo = StreamingAccuracyAlgo(allParam) -// -// val streamingAccuracyProcess = StreamingAccuracyProcess( -// sourceDataConnector, targetDataConnector, -// ruleAnalyzer, cacheResultProcesser, persistFactory, appPersist) -// -// val processInterval = TimeUtil.milliseconds(sparkParam.processInterval) match { +// } +// val cleanInterval = TimeUtil.milliseconds(cleanerParam.cleanInterval) match { // case Some(interval) => interval // case _ => throw new Exception("invalid batch interval") // } -// val process = TimingProcess(processInterval, streamingAccuracyProcess) -// -// // clean thread -//// case class Clean() extends Runnable { -//// val lock = InfoCacheInstance.genLock("clean") -//// def run(): Unit = { -//// val locked = lock.lock(5, TimeUnit.SECONDS) -//// if (locked) { -//// try { -//// sourceDataConnector.cleanData -//// targetDataConnector.cleanData -//// } finally { -//// lock.unlock() -//// } -//// } -//// } -//// } -//// val cleanInterval = TimeUtil.milliseconds(cleanerParam.cleanInterval) match { -//// case Some(interval) => interval -//// case _ => throw new Exception("invalid batch interval") -//// } -//// val clean = TimingProcess(cleanInterval, Clean()) -// -// process.startup() -//// clean.startup() -// -// ssc.start() -// ssc.awaitTermination() -// ssc.stop(stopSparkContext=true, stopGracefully=true) -// -// // context stop -// sc.stop -// -// InfoCacheInstance.close -// -// appPersist.finish() -// -// process.shutdown() -//// clean.shutdown() -// } -// -// private def readParamFile[T <: Param](file: String, fsType: String)(implicit m : Manifest[T]): Try[T] = { -// val paramReader = ParamReaderFactory.getParamReader(file, fsType) -// paramReader.readConfig[T] -// } -// -// private def validateParams(allParam: AllParam): Try[Boolean] = { -// val allParamValidator = AllParamValidator() -// allParamValidator.validate(allParam) -// } -// -//} +// val clean = TimingProcess(cleanInterval, Clean()) + + process.startup() +// clean.startup() + + ssc.start() + ssc.awaitTermination() + ssc.stop(stopSparkContext=true, stopGracefully=true) + + println("================ end ================") + + // context stop + sc.stop + + InfoCacheInstance.close + + appPersist.finish() + + process.shutdown() +// clean.shutdown() + } + + private def readParamFile[T <: Param](file: String, fsType: String)(implicit m : Manifest[T]): Try[T] = { + val paramReader = ParamReaderFactory.getParamReader(file, fsType) + paramReader.readConfig[T] + } + + private def validateParams(allParam: AllParam): Try[Boolean] = { + val allParamValidator = AllParamValidator() + allParamValidator.validate(allParam) + } + +} From d6f8157ed00d0408d490c5f4924ab809d27f1232 Mon Sep 17 00:00:00 2001 From: Lionel Liu Date: Sat, 29 Jul 2017 00:17:01 +0800 Subject: [PATCH 033/111] onboard --- .../streaming/StreamingAccuracyAlgo.scala | 2 + .../streaming/StreamingAccuracyProcess.scala | 27 +- .../connector/DataConnectorFactory.scala | 2 + .../cache/ParquetCacheDataConnector.scala | 370 ++++++++++++++++++ .../measure/persist/LoggerPersist.scala | 4 +- .../griffin/measure/rule/RuleParser.scala | 3 +- .../measure/utils/HdfsFileDumpUtil.scala | 2 +- .../griffin/measure/utils/HdfsUtil.scala | 39 +- .../src/test/resources/config-streaming2.json | 65 +++ .../streaming/StreamingAccuracyAlgoTest.scala | 2 +- .../measure/connector/test/ParquetTest.scala | 131 +++++++ .../griffin/measure/rule/RuleParserTest.scala | 11 + 12 files changed, 635 insertions(+), 23 deletions(-) create mode 100644 measure/src/main/scala/org/apache/griffin/measure/connector/cache/ParquetCacheDataConnector.scala create mode 100644 measure/src/test/resources/config-streaming2.json create mode 100644 measure/src/test/scala/org/apache/griffin/measure/connector/test/ParquetTest.scala diff --git a/measure/src/main/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgo.scala b/measure/src/main/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgo.scala index 31ad18cde..1ecba2936 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgo.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgo.scala @@ -34,6 +34,7 @@ import org.apache.griffin.measure.rule.{ExprValueUtil, RuleAnalyzer, RuleFactory import org.apache.griffin.measure.rule.expr._ import org.apache.griffin.measure.utils.TimeUtil import org.apache.spark.rdd.RDD +import org.apache.spark.sql.SQLContext import org.apache.spark.sql.hive.HiveContext import org.apache.spark.streaming.{Milliseconds, Seconds, StreamingContext} import org.apache.spark.{SparkConf, SparkContext} @@ -55,6 +56,7 @@ case class StreamingAccuracyAlgo(allParam: AllParam) extends AccuracyAlgo { val sc = new SparkContext(conf) sc.setLogLevel(sparkParam.logLevel) val sqlContext = new HiveContext(sc) +// val sqlContext = new SQLContext(sc) val batchInterval = TimeUtil.milliseconds(sparkParam.batchInterval) match { case Some(interval) => Milliseconds(interval) diff --git a/measure/src/main/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyProcess.scala b/measure/src/main/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyProcess.scala index a5ec7d1e4..8545ef9dc 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyProcess.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyProcess.scala @@ -115,6 +115,7 @@ case class StreamingAccuracyProcess(sourceDataConnector: DirectDataConnector, val updateDataPart = updateResults.map(_._2) val updateResultsArray = updateResultsPart.collect() + val updateDataArray = updateDataPart.collect() // update results cache (in driver) // collect action is traversable once action, it will make rdd updateResults empty @@ -126,26 +127,28 @@ case class StreamingAccuracyProcess(sourceDataConnector: DirectDataConnector, persist.result(updateTime, updateResult.result) } - // record missing data and update old data (in executor) - updateDataPart.foreach { grp => + // dump old data (in driver) + updateDataArray.foreach { grp => val (t, datas) = grp - val persist: Persist = persistFactory.getPersists(t) - // persist missing data - val missStrings = datas.map { row => - record2String(row, ruleAnalyzer.sourceRuleExprs.persistExprs, ruleAnalyzer.targetRuleExprs.persistExprs) - } - persist.records(missStrings, PersistType.MISS) // data connector update old data val dumpDatas = datas.map { r => val (_, (v, i)) = r v ++ i } -// println(t) -// dumpDatas.foreach(println) - sourceDataConnector.updateOldData(t, dumpDatas) - targetDataConnector.updateOldData(t, dumpDatas) // not correct +// targetDataConnector.updateOldData(t, dumpDatas) // not correct + } + + // record missing data (in executor) + updateDataPart.foreach { grp => + val (t, datas) = grp + val persist: Persist = persistFactory.getPersists(t) + // persist missing data + val missStrings = datas.map { row => + record2String(row, ruleAnalyzer.sourceRuleExprs.persistExprs, ruleAnalyzer.targetRuleExprs.persistExprs) + } + persist.records(missStrings, PersistType.MISS) } updateResults.unpersist() diff --git a/measure/src/main/scala/org/apache/griffin/measure/connector/DataConnectorFactory.scala b/measure/src/main/scala/org/apache/griffin/measure/connector/DataConnectorFactory.scala index 72456d8b4..d2c11ff8c 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/connector/DataConnectorFactory.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/connector/DataConnectorFactory.scala @@ -40,6 +40,7 @@ object DataConnectorFactory { val KafkaRegex = """^(?i)kafka$""".r val DfRegex = """^(?i)df|dataframe$""".r + val ParquetRegex = """^(?i)parquet$""".r def getDirectDataConnector(sqlContext: SQLContext, ssc: StreamingContext, @@ -91,6 +92,7 @@ object DataConnectorFactory { cacheType match { case DfRegex() => DfCacheDataConnector(sqlContext, dataCacheParam) case HiveRegex() => HiveCacheDataConnector(sqlContext, dataCacheParam) + case ParquetRegex() => ParquetCacheDataConnector(sqlContext, dataCacheParam) case _ => throw new Exception("cache connector creation error!") } } diff --git a/measure/src/main/scala/org/apache/griffin/measure/connector/cache/ParquetCacheDataConnector.scala b/measure/src/main/scala/org/apache/griffin/measure/connector/cache/ParquetCacheDataConnector.scala new file mode 100644 index 000000000..4cd276564 --- /dev/null +++ b/measure/src/main/scala/org/apache/griffin/measure/connector/cache/ParquetCacheDataConnector.scala @@ -0,0 +1,370 @@ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ +package org.apache.griffin.measure.connector.cache + +import java.util.concurrent.TimeUnit + +import org.apache.griffin.measure.cache.info.{InfoCacheInstance, TimeInfoCache} +import org.apache.griffin.measure.config.params.user.DataCacheParam +import org.apache.griffin.measure.result.TimeStampInfo +import org.apache.griffin.measure.utils.{HdfsUtil, JsonUtil, TimeUtil} +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.{Row, SQLContext} +import org.apache.spark.sql.types._ +import scala.collection.JavaConversions._ + +import scala.util.Try + +case class ParquetCacheDataConnector(sqlContext: SQLContext, dataCacheParam: DataCacheParam + ) extends CacheDataConnector { + + val config = dataCacheParam.config + val InfoPath = "info.path" + val cacheInfoPath: String = config.getOrElse(InfoPath, defCacheInfoPath).toString + + val newCacheLock = InfoCacheInstance.genLock(s"${cacheInfoPath}.new") + val oldCacheLock = InfoCacheInstance.genLock(s"${cacheInfoPath}.old") + + val timeRangeParam: List[String] = if (dataCacheParam.timeRange != null) dataCacheParam.timeRange else Nil + val deltaTimeRange: (Long, Long) = (timeRangeParam ::: List("0", "0")) match { + case s :: e :: _ => { + val ns = TimeUtil.milliseconds(s) match { + case Some(n) if (n < 0) => n + case _ => 0 + } + val ne = TimeUtil.milliseconds(e) match { + case Some(n) if (n < 0) => n + case _ => 0 + } + (ns, ne) + } + case _ => (0, 0) + } + + val FilePath = "file.path" + val filePath: String = config.get(FilePath) match { + case Some(s: String) => s + case _ => throw new Exception("invalid file.path!") + } + + val ReadyTimeInterval = "ready.time.interval" + val ReadyTimeDelay = "ready.time.delay" + val readyTimeInterval: Long = TimeUtil.milliseconds(config.getOrElse(ReadyTimeInterval, "1m").toString).getOrElse(60000L) + val readyTimeDelay: Long = TimeUtil.milliseconds(config.getOrElse(ReadyTimeDelay, "1m").toString).getOrElse(60000L) + + val TimeStampColumn: String = TimeStampInfo.key + val PayloadColumn: String = "payload" + + // cache schema: Long, String + val fields = List[StructField]( + StructField(TimeStampColumn, LongType), + StructField(PayloadColumn, StringType) + ) + val schema = StructType(fields) + +// case class CacheData(time: Long, payload: String) { +// def getTime(): Long = time +// def getPayload(): String = payload +// } + + val partitionUnits: List[String] = List("hour", "min") + + override def init(): Unit = { + // do nothing + } + + def available(): Boolean = { + true + } + + private def encode(data: Map[String, Any], ms: Long): Option[Row] = { + try { + val values = fields.map { field => + val StructField(name, _, _, _) = field + name match { + case TimeStampColumn => ms + case PayloadColumn => JsonUtil.toJson(data) + case _ => null + } + } + Some(Row(values: _*)) + } catch { + case _ => None + } + } + + private def decode(data: Row, updateTimeStamp: Boolean): Option[Map[String, Any]] = { + data.getAs[Any](PayloadColumn) match { + case v: String => { + try { + val map = JsonUtil.toAnyMap(v) + val resMap = if (updateTimeStamp) { + data.getAs[Any](TimeStampColumn) match { + case t: Long => map + (TimeStampColumn -> t) + case _ => map + } + } else map + Some(resMap) + } catch { + case _ => None + } + } + case _ => None + } + } + +// private def encode(data: Map[String, Any], ms: Long): Option[CacheData] = { +// try { +// val json = JsonUtil.toJson(data) +// Some(CacheData(ms, json)) +// } catch { +// case _ => None +// } +// } +// +// private def decode(data: Row, updateTimeStamp: Boolean): Option[Map[String, Any]] = { +// data.getAs[Any]("payload") match { +// case v: String => { +// try { +// val map = JsonUtil.toAnyMap(v) +// val resMap = if (updateTimeStamp) { +// data.getAs[Any]("time") match { +// case t: Long => map + (TimeStampInfo.key -> t) +// case _ => map +// } +// } else map +// Some(resMap) +// } catch { +// case _ => None +// } +// } +// case _ => None +// } +// } + + def saveData(rdd: RDD[Map[String, Any]], ms: Long): Unit = { + val newCacheLocked = newCacheLock.lock(-1, TimeUnit.SECONDS) + if (newCacheLocked) { + try { + if (!rdd.isEmpty) { + val ptns = getPartition(ms) + val ptnsPath = genPartitionHdfsPath(ptns) + val dirPath = s"${filePath}/${ptnsPath}" + val dataFileName = s"${ms}" + val dataFilePath = HdfsUtil.getHdfsFilePath(dirPath, dataFileName) + + // encode data + val dataRdd: RDD[Row] = rdd.flatMap(encode(_, ms)) +// val dataRdd: RDD[CacheData] = rdd.flatMap(encode(_, ms)) + + // generate data frame + val df = sqlContext.createDataFrame(dataRdd, schema) +// val df = sqlContext.createDataFrame(dataRdd) + + // save data frame + df.write.parquet(dataFilePath) + } + // submit ms + submitCacheTime(ms) + submitReadyTime(ms) + } catch { + case e: Throwable => error(s"save data error: ${e.getMessage}") + } finally { + newCacheLock.unlock() + } + } + } + + def readData(): Try[RDD[Map[String, Any]]] = Try { + val timeRange = TimeInfoCache.getTimeRange + submitLastProcTime(timeRange._2) + + val reviseTimeRange = (timeRange._1 + deltaTimeRange._1, timeRange._2 + deltaTimeRange._2) + submitCleanTime(reviseTimeRange._1) + + // read directly through partition info + val partitionRanges = getPartitionRange(reviseTimeRange._1, reviseTimeRange._2) + val alterPartitionRanges = partitionRanges :+ (reviseTimeRange._1, reviseTimeRange._2) + println(alterPartitionRanges) + + // list partition paths + val partitionPaths = listPathsBetweenRanges(filePath :: Nil, alterPartitionRanges) + + if (partitionPaths.isEmpty) { + sqlContext.sparkContext.emptyRDD[Map[String, Any]] + } else { + val df = partitionPaths.map { path => + sqlContext.read.parquet(path) + }.reduce(_ unionAll _) + + // decode data + df.flatMap { row => + decode(row, true) + } + } + } + + override def cleanOldData(): Unit = { + val oldCacheLocked = oldCacheLock.lock(-1, TimeUnit.SECONDS) + if (oldCacheLocked) { + try { + val cleanTime = readCleanTime() + cleanTime match { + case Some(ct) => { + // drop partitions + val bounds = getPartition(ct) + val alterBounds = bounds :+ ct + + // list partition paths + val earlierPaths = listPathsEarlierThanBounds(filePath :: Nil, alterBounds) + + // delete out time data path + earlierPaths.foreach { path => + println(s"delete hdfs path: ${path}") + HdfsUtil.deleteHdfsPath(path) + } + } + case _ => { + // do nothing + } + } + } catch { + case e: Throwable => error(s"clean old data error: ${e.getMessage}") + } finally { + oldCacheLock.unlock() + } + } + } + + override def updateOldData(t: Long, oldData: Iterable[Map[String, Any]]): Unit = { + // parallel process different time groups, lock is unnecessary + val ptns = getPartition(t) + val ptnsPath = genPartitionHdfsPath(ptns) + val dirPath = s"${filePath}/${ptnsPath}" + val dataFileName = s"${t}" + val dataFilePath = HdfsUtil.getHdfsFilePath(dirPath, dataFileName) + + try { + // remove old data path + HdfsUtil.deleteHdfsPath(dataFilePath) + + // save updated old data + if (oldData.size > 0) { + // encode data + val recordDatas = oldData.flatMap { dt => + encode(dt, t) + }.toList + +// val rdd = sqlContext.sparkContext.parallelize(recordDatas) + + // generate data frame + val df = sqlContext.createDataFrame(recordDatas, schema) + + // save data frame + df.write.parquet(dataFilePath) + } + } catch { + case e: Throwable => { + error(s"update old data error: ${e.getMessage}") + e.printStackTrace() + } + } + } + + override protected def genCleanTime(ms: Long): Long = { + val minPartitionUnit = partitionUnits.last + val t1 = TimeUtil.timeToUnit(ms, minPartitionUnit) + val t2 = TimeUtil.timeFromUnit(t1, minPartitionUnit) + t2 + } + + private def getPartition(ms: Long): List[Long] = { + partitionUnits.map { unit => + TimeUtil.timeToUnit(ms, unit) + } + } + private def getPartitionRange(ms1: Long, ms2: Long): List[(Long, Long)] = { + partitionUnits.map { unit => + val t1 = TimeUtil.timeToUnit(ms1, unit) + val t2 = TimeUtil.timeToUnit(ms2, unit) + (t1, t2) + } + } + + private def genPartitionHdfsPath(partition: List[Long]): String = { + partition.map(prtn => s"${prtn}").mkString("/") + } + + private def str2Long(str: String): Option[Long] = { + try { + Some(str.toLong) + } catch { + case e: Throwable => None + } + } + + private def listPathsBetweenRanges(paths: List[String], + partitionRanges: List[(Long, Long)] + ): List[String] = { + partitionRanges match { + case Nil => paths + case head :: tail => { + val (lb, ub) = head + val curPaths = paths.flatMap { path => + val names = HdfsUtil.listSubPaths(path, "dir").toList + names.filter { name => + str2Long(name) match { + case Some(t) => (t >= lb) && (t <= ub) + case _ => false + } + }.map(HdfsUtil.getHdfsFilePath(path, _)) + } + listPathsBetweenRanges(curPaths, tail) + } + } + } + + private def listPathsEarlierThanBounds(paths: List[String], bounds: List[Long] + ): List[String] = { + bounds match { + case Nil => paths + case head :: tail => { + val earlierPaths = paths.flatMap { path => + val names = HdfsUtil.listSubPaths(path, "dir").toList + names.filter { name => + str2Long(name) match { + case Some(t) => (t < head) + case _ => false + } + }.map(HdfsUtil.getHdfsFilePath(path, _)) + } + val equalPaths = paths.flatMap { path => + val names = HdfsUtil.listSubPaths(path, "dir").toList + names.filter { name => + str2Long(name) match { + case Some(t) => (t == head) + case _ => false + } + }.map(HdfsUtil.getHdfsFilePath(path, _)) + } + earlierPaths ::: listPathsEarlierThanBounds(equalPaths, tail) + } + } + } +} diff --git a/measure/src/main/scala/org/apache/griffin/measure/persist/LoggerPersist.scala b/measure/src/main/scala/org/apache/griffin/measure/persist/LoggerPersist.scala index a066ddf85..00d41ea28 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/persist/LoggerPersist.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/persist/LoggerPersist.scala @@ -66,7 +66,7 @@ case class LoggerPersist(config: Map[String, Any], metricName: String, timeStamp val count = if (maxLogLines < 0) recordCount else scala.math.min(maxLogLines, recordCount) if (count > 0) { val recordsArray = records.take(count) - recordsArray.foreach(println) +// recordsArray.foreach(println) } } catch { case e: Throwable => error(e.getMessage) @@ -79,7 +79,7 @@ case class LoggerPersist(config: Map[String, Any], metricName: String, timeStamp val count = if (maxLogLines < 0) recordCount else scala.math.min(maxLogLines, recordCount) if (count > 0) { val recordsArray = records.take(count) - recordsArray.foreach(println) +// recordsArray.foreach(println) } } catch { case e: Throwable => error(e.getMessage) diff --git a/measure/src/main/scala/org/apache/griffin/measure/rule/RuleParser.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/RuleParser.scala index 99831f2a1..55d9f4591 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rule/RuleParser.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rule/RuleParser.scala @@ -125,7 +125,8 @@ case class RuleParser() extends JavaTokenParsers with Serializable { import Operator._ object SomeString { - def AnyString: Parser[String] = """[^'\"{}\[\]()=<>.$@,;+\-*/\\]*""".r +// def AnyString: Parser[String] = """[^'\"{}\[\]()=<>.$@,;+\-*/\\]*""".r + def AnyString: Parser[String] = """[^'\"]*""".r def SimpleFieldString: Parser[String] = """\w+""".r def FieldString: Parser[String] = """[\w\s]+""".r def NameString: Parser[String] = """[a-zA-Z_]\w*""".r diff --git a/measure/src/main/scala/org/apache/griffin/measure/utils/HdfsFileDumpUtil.scala b/measure/src/main/scala/org/apache/griffin/measure/utils/HdfsFileDumpUtil.scala index fc3860f8a..8a608ff0b 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/utils/HdfsFileDumpUtil.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/utils/HdfsFileDumpUtil.scala @@ -68,7 +68,7 @@ object HdfsFileDumpUtil { def remove(path: String, filename: String, withSuffix: Boolean): Unit = { if (withSuffix) { - val files = HdfsUtil.listPathFiles(path) + val files = HdfsUtil.listSubPaths(path, "file") val patternFiles = files.filter(samePattern(_, filename)) patternFiles.foreach { f => val rmPath = HdfsUtil.getHdfsFilePath(path, f) diff --git a/measure/src/main/scala/org/apache/griffin/measure/utils/HdfsUtil.scala b/measure/src/main/scala/org/apache/griffin/measure/utils/HdfsUtil.scala index c97a2112b..6dd54b7f6 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/utils/HdfsUtil.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/utils/HdfsUtil.scala @@ -27,7 +27,7 @@ object HdfsUtil { private val conf = new Configuration() conf.set("dfs.support.append", "true") - conf.set("fs.defaultFS", "hdfs://localhost") // debug @localhost +// conf.set("fs.defaultFS", "hdfs://localhost") // debug @localhost private val dfs = FileSystem.get(conf) @@ -79,14 +79,41 @@ object HdfsUtil { if (dfs.exists(path)) dfs.delete(path, true) } - def listPathFiles(dirPath: String): Iterable[String] = { +// def listPathFiles(dirPath: String): Iterable[String] = { +// val path = new Path(dirPath) +// try { +// val fileStatusArray = dfs.listStatus(path) +// fileStatusArray.flatMap { fileStatus => +// if (fileStatus.isFile) { +// Some(fileStatus.getPath.getName) +// } else None +// } +// } catch { +// case e: Throwable => { +// println(s"list path files error: ${e.getMessage}") +// Nil +// } +// } +// } + + def listSubPaths(dirPath: String, subType: String, fullPath: Boolean = false): Iterable[String] = { val path = new Path(dirPath) try { val fileStatusArray = dfs.listStatus(path) - fileStatusArray.flatMap { fileStatus => - if (fileStatus.isFile) { - Some(fileStatus.getPath.getName) - } else None + fileStatusArray.filter { fileStatus => + subType match { + case "dir" => fileStatus.isDirectory + case "file" => fileStatus.isFile + case _ => true + } + }.map { fileStatus => + val fname = fileStatus.getPath.getName + if (fullPath) getHdfsFilePath(dirPath, fname) else fname + } + } catch { + case e: Throwable => { + println(s"list path files error: ${e.getMessage}") + Nil } } } diff --git a/measure/src/test/resources/config-streaming2.json b/measure/src/test/resources/config-streaming2.json new file mode 100644 index 000000000..061382b65 --- /dev/null +++ b/measure/src/test/resources/config-streaming2.json @@ -0,0 +1,65 @@ +{ + "name": "accu1", + "type": "accuracy", + + "process.type": "streaming", + + "source": { + "type": "kafka", + "version": "0.8", + "config": { + "kafka.config": { + "bootstrap.servers": "10.149.247.156:9092", + "group.id": "group1", + "auto.offset.reset": "smallest", + "auto.commit.enable": "false" + }, + "topics": "sss", + "key.type": "java.lang.String", + "value.type": "java.lang.String" + }, + "cache": { + "type": "parquet", + "config": { + "file.path": "hdfs://localhost/griffin/streaming/dump/source", + "info.path": "source", + "ready.time.interval": "10s", + "ready.time.delay": "0" + }, + "time.range": ["-2m", "0"] + }, + "match.once": true + }, + + "target": { + "type": "kafka", + "version": "0.8", + "config": { + "kafka.config": { + "bootstrap.servers": "10.149.247.156:9092", + "group.id": "group1", + "auto.offset.reset": "smallest", + "auto.commit.enable": "false" + }, + "topics": "ttt", + "key.type": "java.lang.String", + "value.type": "java.lang.String" + }, + "cache": { + "type": "parquet", + "config": { + "file.path": "hdfs://localhost/griffin/streaming/dump/target", + "info.path": "target", + "ready.time.interval": "10s", + "ready.time.delay": "0" + }, + "time.range": ["-2m", "0"] + }, + "match.once": false + }, + + "evaluateRule": { + "sampleRatio": 0.2, + "rules": "$source.json().name = $target.json().name AND $source.json().age = $target.json().age" + } +} \ No newline at end of file diff --git a/measure/src/test/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgoTest.scala b/measure/src/test/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgoTest.scala index d92da30c1..be8735a05 100644 --- a/measure/src/test/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgoTest.scala +++ b/measure/src/test/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgoTest.scala @@ -53,7 +53,7 @@ import scala.util.{Failure, Success, Try} class StreamingAccuracyAlgoTest extends FunSuite with Matchers with BeforeAndAfter with Loggable { val envFile = "src/test/resources/env-streaming.json" - val confFile = "src/test/resources/config-streaming.json" + val confFile = "src/test/resources/config-streaming2.json" val envFsType = "local" val userFsType = "local" diff --git a/measure/src/test/scala/org/apache/griffin/measure/connector/test/ParquetTest.scala b/measure/src/test/scala/org/apache/griffin/measure/connector/test/ParquetTest.scala new file mode 100644 index 000000000..5bdc17556 --- /dev/null +++ b/measure/src/test/scala/org/apache/griffin/measure/connector/test/ParquetTest.scala @@ -0,0 +1,131 @@ +package org.apache.griffin.measure.connector.test + + +import java.util.Date + +import org.apache.griffin.measure.rule.DataTypeCalculationUtil +import org.apache.griffin.measure.utils.HdfsUtil +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.{DataFrame, Row, SQLContext} +import org.apache.spark.sql.hive.HiveContext +import org.apache.spark.sql.types.{DataType, StructField, StructType} +import org.apache.spark.{SparkConf, SparkContext} +import org.junit.runner.RunWith +import org.scalatest.junit.JUnitRunner +import org.scalatest.{BeforeAndAfter, FunSuite, Matchers} + + +@RunWith(classOf[JUnitRunner]) +class ParquetTest extends FunSuite with Matchers with BeforeAndAfter { + + test ("parquet") { + val conf = new SparkConf().setMaster("local[*]").setAppName("parquet") + val sc = new SparkContext(conf) +// sc.setLogLevel(envParam.sparkParam.logLevel) + val sqlContext = new SQLContext(sc) +// sqlContext = new HiveContext(sc) + + val t1 = new Date().getTime() + +// val data = (0 to 99999).toList.map(d => Map[String, Any]( +// ("name" -> s"s${d}"), +// ("age" -> (d % 10)) +// )) + + case class Data(name: String, age: Long) { + def getName = name + def getAge = age + } + val data = (0 to 99999).toList.map(d => Data(s"s${d}", d % 10)) + + val rdd = sc.parallelize(data) + +// val df = genDataFrame(sqlContext, rdd) + val df = sqlContext.createDataFrame(rdd, classOf[Data]) + println(df.count) + + val t2 = new Date().getTime() + + df.write.partitionBy("age").parquet("hdfs://localhost/test/parq") + + val t3 = new Date().getTime() + println(s"write time: ${t3 - t2}") + + val readDf = sqlContext.read.parquet("hdfs://localhost/test/parq") + readDf.show() + println(readDf.count) + + val t4 = new Date().getTime() + println(s"read time: ${t4 - t3}") + + } + + private def genDataFrame(sqlContext: SQLContext, rdd: RDD[Map[String, Any]]): DataFrame = { + val fields = rdd.aggregate(Map[String, DataType]())( + DataTypeCalculationUtil.sequenceDataTypeMap, DataTypeCalculationUtil.combineDataTypeMap + ).toList.map(f => StructField(f._1, f._2)) + val schema = StructType(fields) + val datas: RDD[Row] = rdd.map { d => + val values = fields.map { field => + val StructField(k, dt, _, _) = field + d.get(k) match { + case Some(v) => v + case _ => null + } + } + Row(values: _*) + } + val df = sqlContext.createDataFrame(datas, schema) + df + } + + test ("list") { + val conf = new SparkConf().setMaster("local[*]").setAppName("parquet") + val sc = new SparkContext(conf) + // sc.setLogLevel(envParam.sparkParam.logLevel) + val sqlContext = new SQLContext(sc) + + val filePath = "hdfs://localhost/griffin/streaming/dump/source" + + val partitionRanges: List[(Long, Long)] = List((417007L,417007L), (25020456L,25020459L)) + + println(partitionRanges) + + // list partition paths + val partitionPaths = listPartitionPathsByRanges(filePath :: Nil, partitionRanges) + + partitionPaths.foreach(println) + } + + private def listPartitionPathsByRanges(paths: List[String], partitionRanges: List[(Long, Long)] + ): List[String] = { + partitionRanges match { + case Nil => paths + case head :: tail => { + val curPaths = listPartitionPathsByRange(paths, head) + listPartitionPathsByRanges(curPaths, tail) + } + } + } + + private def listPartitionPathsByRange(paths: List[String], partitionRange: (Long, Long) + ): List[String] = { + val (lb, ub) = partitionRange + paths.flatMap { path => + val names = HdfsUtil.listSubPaths(path, "dir").toList + names.filter { name => + val t = str2Long(name) + (t > 0) && (t >= lb) && (t <= ub) + }.map(HdfsUtil.getHdfsFilePath(path, _)) + } + } + + private def str2Long(str: String): Long = { + try { + str.toLong + } catch { + case e: Throwable => -1 + } + } + +} diff --git a/measure/src/test/scala/org/apache/griffin/measure/rule/RuleParserTest.scala b/measure/src/test/scala/org/apache/griffin/measure/rule/RuleParserTest.scala index afcb95d25..1d153757d 100644 --- a/measure/src/test/scala/org/apache/griffin/measure/rule/RuleParserTest.scala +++ b/measure/src/test/scala/org/apache/griffin/measure/rule/RuleParserTest.scala @@ -51,6 +51,11 @@ class RuleParserTest extends FunSuite with Matchers with BeforeAndAfter { val result2 = ruleParser.parseAll(ruleParser.literal, rule1) result2.successful should be (true) result2.get.value should be (Some("123")) + + val rule3 = "'1+2-3'" + val result3 = ruleParser.parseAll(ruleParser.literal, rule3) + result3.successful should be (true) + result3.get.value should be (Some("1+2-3")) } test ("literial time") { @@ -155,6 +160,12 @@ class RuleParserTest extends FunSuite with Matchers with BeforeAndAfter { val result = ruleParser.parseAll(ruleParser.mathExpr, rule) result.successful should be (true) result.get.desc should be ("$source['age'] * 6 + 4 / 2") + + val rule2 = "'age + 1' / 'vv'" + val result2 = ruleParser.parseAll(ruleParser.mathExpr, rule2) + result2.successful should be (true) + result2.get.desc should be ("'age + 1' / 'vv'") + println(result2) } test ("range expr") { From 02e490004fa36b6563a95c82118b785b8b7d0bfb Mon Sep 17 00:00:00 2001 From: Lionel Liu Date: Sun, 30 Jul 2017 12:30:14 +0800 Subject: [PATCH 034/111] v3 --- .../streaming/StreamingAccuracyProcess.scala | 24 +++---- .../cache/ParquetCacheDataConnector.scala | 68 +++++++++---------- .../src/test/resources/config-streaming3.json | 65 ++++++++++++++++++ 3 files changed, 108 insertions(+), 49 deletions(-) create mode 100644 measure/src/test/resources/config-streaming3.json diff --git a/measure/src/main/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyProcess.scala b/measure/src/main/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyProcess.scala index 8545ef9dc..dafcb2662 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyProcess.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyProcess.scala @@ -115,7 +115,6 @@ case class StreamingAccuracyProcess(sourceDataConnector: DirectDataConnector, val updateDataPart = updateResults.map(_._2) val updateResultsArray = updateResultsPart.collect() - val updateDataArray = updateDataPart.collect() // update results cache (in driver) // collect action is traversable once action, it will make rdd updateResults empty @@ -127,20 +126,7 @@ case class StreamingAccuracyProcess(sourceDataConnector: DirectDataConnector, persist.result(updateTime, updateResult.result) } - // dump old data (in driver) - updateDataArray.foreach { grp => - val (t, datas) = grp - // data connector update old data - val dumpDatas = datas.map { r => - val (_, (v, i)) = r - v ++ i - } - - sourceDataConnector.updateOldData(t, dumpDatas) -// targetDataConnector.updateOldData(t, dumpDatas) // not correct - } - - // record missing data (in executor) + // record missing data and dump old data (in executor) updateDataPart.foreach { grp => val (t, datas) = grp val persist: Persist = persistFactory.getPersists(t) @@ -149,6 +135,14 @@ case class StreamingAccuracyProcess(sourceDataConnector: DirectDataConnector, record2String(row, ruleAnalyzer.sourceRuleExprs.persistExprs, ruleAnalyzer.targetRuleExprs.persistExprs) } persist.records(missStrings, PersistType.MISS) + + // data connector update old data + val dumpDatas = datas.map { r => + val (_, (v, i)) = r + v ++ i + } + sourceDataConnector.updateOldData(t, dumpDatas) + // targetDataConnector.updateOldData(t, dumpDatas) // not correct } updateResults.unpersist() diff --git a/measure/src/main/scala/org/apache/griffin/measure/connector/cache/ParquetCacheDataConnector.scala b/measure/src/main/scala/org/apache/griffin/measure/connector/cache/ParquetCacheDataConnector.scala index 4cd276564..b641d22fe 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/connector/cache/ParquetCacheDataConnector.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/connector/cache/ParquetCacheDataConnector.scala @@ -252,40 +252,40 @@ case class ParquetCacheDataConnector(sqlContext: SQLContext, dataCacheParam: Dat } } - override def updateOldData(t: Long, oldData: Iterable[Map[String, Any]]): Unit = { - // parallel process different time groups, lock is unnecessary - val ptns = getPartition(t) - val ptnsPath = genPartitionHdfsPath(ptns) - val dirPath = s"${filePath}/${ptnsPath}" - val dataFileName = s"${t}" - val dataFilePath = HdfsUtil.getHdfsFilePath(dirPath, dataFileName) - - try { - // remove old data path - HdfsUtil.deleteHdfsPath(dataFilePath) - - // save updated old data - if (oldData.size > 0) { - // encode data - val recordDatas = oldData.flatMap { dt => - encode(dt, t) - }.toList - -// val rdd = sqlContext.sparkContext.parallelize(recordDatas) - - // generate data frame - val df = sqlContext.createDataFrame(recordDatas, schema) - - // save data frame - df.write.parquet(dataFilePath) - } - } catch { - case e: Throwable => { - error(s"update old data error: ${e.getMessage}") - e.printStackTrace() - } - } - } +// override def updateOldData(t: Long, oldData: Iterable[Map[String, Any]]): Unit = { +// // parallel process different time groups, lock is unnecessary +// val ptns = getPartition(t) +// val ptnsPath = genPartitionHdfsPath(ptns) +// val dirPath = s"${filePath}/${ptnsPath}" +// val dataFileName = s"${t}" +// val dataFilePath = HdfsUtil.getHdfsFilePath(dirPath, dataFileName) +// +// try { +// // remove old data path +// HdfsUtil.deleteHdfsPath(dataFilePath) +// +// // save updated old data +// if (oldData.size > 0) { +// // encode data +// val recordDatas = oldData.flatMap { dt => +// encode(dt, t) +// }.toList +// +//// val rdd = sqlContext.sparkContext.parallelize(recordDatas) +// +// // generate data frame +// val df = sqlContext.createDataFrame(recordDatas, schema) +// +// // save data frame +// df.write.parquet(dataFilePath) +// } +// } catch { +// case e: Throwable => { +// error(s"update old data error: ${e.getMessage}") +// e.printStackTrace() +// } +// } +// } override protected def genCleanTime(ms: Long): Long = { val minPartitionUnit = partitionUnits.last diff --git a/measure/src/test/resources/config-streaming3.json b/measure/src/test/resources/config-streaming3.json new file mode 100644 index 000000000..061382b65 --- /dev/null +++ b/measure/src/test/resources/config-streaming3.json @@ -0,0 +1,65 @@ +{ + "name": "accu1", + "type": "accuracy", + + "process.type": "streaming", + + "source": { + "type": "kafka", + "version": "0.8", + "config": { + "kafka.config": { + "bootstrap.servers": "10.149.247.156:9092", + "group.id": "group1", + "auto.offset.reset": "smallest", + "auto.commit.enable": "false" + }, + "topics": "sss", + "key.type": "java.lang.String", + "value.type": "java.lang.String" + }, + "cache": { + "type": "parquet", + "config": { + "file.path": "hdfs://localhost/griffin/streaming/dump/source", + "info.path": "source", + "ready.time.interval": "10s", + "ready.time.delay": "0" + }, + "time.range": ["-2m", "0"] + }, + "match.once": true + }, + + "target": { + "type": "kafka", + "version": "0.8", + "config": { + "kafka.config": { + "bootstrap.servers": "10.149.247.156:9092", + "group.id": "group1", + "auto.offset.reset": "smallest", + "auto.commit.enable": "false" + }, + "topics": "ttt", + "key.type": "java.lang.String", + "value.type": "java.lang.String" + }, + "cache": { + "type": "parquet", + "config": { + "file.path": "hdfs://localhost/griffin/streaming/dump/target", + "info.path": "target", + "ready.time.interval": "10s", + "ready.time.delay": "0" + }, + "time.range": ["-2m", "0"] + }, + "match.once": false + }, + + "evaluateRule": { + "sampleRatio": 0.2, + "rules": "$source.json().name = $target.json().name AND $source.json().age = $target.json().age" + } +} \ No newline at end of file From 5958eb3b851c6aaa7d3776e9d254c5808f31bf2c Mon Sep 17 00:00:00 2001 From: Lionel Liu Date: Sun, 30 Jul 2017 21:39:33 +0800 Subject: [PATCH 035/111] v4 --- .../streaming/StreamingAccuracyProcess.scala | 9 +- .../connector/DataConnectorFactory.scala | 2 + .../cache/TextCacheDataConnector.scala | 311 ++++++++++++++++++ .../measure/persist/OldHttpPersist.scala | 87 +++++ .../measure/persist/PersistFactory.scala | 2 + .../src/test/resources/config-streaming3.json | 4 +- .../streaming/StreamingAccuracyAlgoTest.scala | 6 +- 7 files changed, 413 insertions(+), 8 deletions(-) create mode 100644 measure/src/main/scala/org/apache/griffin/measure/connector/cache/TextCacheDataConnector.scala create mode 100644 measure/src/main/scala/org/apache/griffin/measure/persist/OldHttpPersist.scala diff --git a/measure/src/main/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyProcess.scala b/measure/src/main/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyProcess.scala index dafcb2662..7f1e63e2e 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyProcess.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyProcess.scala @@ -46,8 +46,9 @@ case class StreamingAccuracyProcess(sourceDataConnector: DirectDataConnector, def run(): Unit = { // println(s"cache count: ${cacheResultProcesser.cacheGroup.size}") - val updateTime = new Date().getTime - println(s"===== [${updateTime}] process begins =====") + val updateTimeDate = new Date() + val updateTime = updateTimeDate.getTime + println(s"===== [${updateTimeDate}] process begins =====") val locked = lock.lock(5, TimeUnit.SECONDS) if (locked) { try { @@ -168,9 +169,11 @@ case class StreamingAccuracyProcess(sourceDataConnector: DirectDataConnector, } finally { lock.unlock() } + } else { + println(s"===== [${updateTimeDate}] process ignores =====") } val endTime = new Date().getTime - println(s"===== [${updateTime}] process ends, using ${endTime - updateTime} ms =====") + println(s"===== [${updateTimeDate}] process ends, using ${endTime - updateTime} ms =====") } // clean old data and old result cache diff --git a/measure/src/main/scala/org/apache/griffin/measure/connector/DataConnectorFactory.scala b/measure/src/main/scala/org/apache/griffin/measure/connector/DataConnectorFactory.scala index d2c11ff8c..517695ced 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/connector/DataConnectorFactory.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/connector/DataConnectorFactory.scala @@ -41,6 +41,7 @@ object DataConnectorFactory { val DfRegex = """^(?i)df|dataframe$""".r val ParquetRegex = """^(?i)parquet$""".r + val TextRegex = """^(?i)text$""".r def getDirectDataConnector(sqlContext: SQLContext, ssc: StreamingContext, @@ -93,6 +94,7 @@ object DataConnectorFactory { case DfRegex() => DfCacheDataConnector(sqlContext, dataCacheParam) case HiveRegex() => HiveCacheDataConnector(sqlContext, dataCacheParam) case ParquetRegex() => ParquetCacheDataConnector(sqlContext, dataCacheParam) + case TextRegex() => TextCacheDataConnector(sqlContext, dataCacheParam) case _ => throw new Exception("cache connector creation error!") } } diff --git a/measure/src/main/scala/org/apache/griffin/measure/connector/cache/TextCacheDataConnector.scala b/measure/src/main/scala/org/apache/griffin/measure/connector/cache/TextCacheDataConnector.scala new file mode 100644 index 000000000..62b608618 --- /dev/null +++ b/measure/src/main/scala/org/apache/griffin/measure/connector/cache/TextCacheDataConnector.scala @@ -0,0 +1,311 @@ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ +package org.apache.griffin.measure.connector.cache + +import java.util.concurrent.TimeUnit + +import org.apache.griffin.measure.cache.info.{InfoCacheInstance, TimeInfoCache} +import org.apache.griffin.measure.config.params.user.DataCacheParam +import org.apache.griffin.measure.result.TimeStampInfo +import org.apache.griffin.measure.utils.{HdfsFileDumpUtil, HdfsUtil, JsonUtil, TimeUtil} +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.SQLContext + +import scala.util.Try + +case class TextCacheDataConnector(sqlContext: SQLContext, dataCacheParam: DataCacheParam + ) extends CacheDataConnector { + + val config = dataCacheParam.config + val InfoPath = "info.path" + val cacheInfoPath: String = config.getOrElse(InfoPath, defCacheInfoPath).toString + + val newCacheLock = InfoCacheInstance.genLock(s"${cacheInfoPath}.new") + val oldCacheLock = InfoCacheInstance.genLock(s"${cacheInfoPath}.old") + + val timeRangeParam: List[String] = if (dataCacheParam.timeRange != null) dataCacheParam.timeRange else Nil + val deltaTimeRange: (Long, Long) = (timeRangeParam ::: List("0", "0")) match { + case s :: e :: _ => { + val ns = TimeUtil.milliseconds(s) match { + case Some(n) if (n < 0) => n + case _ => 0 + } + val ne = TimeUtil.milliseconds(e) match { + case Some(n) if (n < 0) => n + case _ => 0 + } + (ns, ne) + } + case _ => (0, 0) + } + + val FilePath = "file.path" + val filePath: String = config.get(FilePath) match { + case Some(s: String) => s + case _ => throw new Exception("invalid file.path!") + } + + val ReadyTimeInterval = "ready.time.interval" + val ReadyTimeDelay = "ready.time.delay" + val readyTimeInterval: Long = TimeUtil.milliseconds(config.getOrElse(ReadyTimeInterval, "1m").toString).getOrElse(60000L) + val readyTimeDelay: Long = TimeUtil.milliseconds(config.getOrElse(ReadyTimeDelay, "1m").toString).getOrElse(60000L) + +// val TimeStampColumn: String = TimeStampInfo.key +// val PayloadColumn: String = "payload" + + // cache schema: Long, String +// val fields = List[StructField]( +// StructField(TimeStampColumn, LongType), +// StructField(PayloadColumn, StringType) +// ) +// val schema = StructType(fields) + + // case class CacheData(time: Long, payload: String) { + // def getTime(): Long = time + // def getPayload(): String = payload + // } + + private val rowSepLiteral = "\n" + + val partitionUnits: List[String] = List("hour", "min") + + override def init(): Unit = { + // do nothing + } + + def available(): Boolean = { + true + } + + private def encode(data: Map[String, Any], ms: Long): Option[String] = { + try { + val map = data + (TimeStampInfo.key -> ms) + Some(JsonUtil.toJson(map)) + } catch { + case _: Throwable => None + } + } + + private def decode(data: String): Option[Map[String, Any]] = { + try { + Some(JsonUtil.toAnyMap(data)) + } catch { + case _: Throwable => None + } + } + + def saveData(rdd: RDD[Map[String, Any]], ms: Long): Unit = { + val newCacheLocked = newCacheLock.lock(-1, TimeUnit.SECONDS) + if (newCacheLocked) { + try { + val ptns = getPartition(ms) + val ptnsPath = genPartitionHdfsPath(ptns) + val dirPath = s"${filePath}/${ptnsPath}" + val dataFileName = s"${ms}" + val dataFilePath = HdfsUtil.getHdfsFilePath(dirPath, dataFileName) + + // encode data + val dataRdd: RDD[String] = rdd.flatMap(encode(_, ms)) + + // save data + val dumped = if (!dataRdd.isEmpty) { + HdfsFileDumpUtil.dump(dataFilePath, dataRdd, rowSepLiteral) + } else false + + // submit ms + submitCacheTime(ms) + submitReadyTime(ms) + } catch { + case e: Throwable => error(s"save data error: ${e.getMessage}") + } finally { + newCacheLock.unlock() + } + } + } + + def readData(): Try[RDD[Map[String, Any]]] = Try { + val timeRange = TimeInfoCache.getTimeRange + submitLastProcTime(timeRange._2) + + val reviseTimeRange = (timeRange._1 + deltaTimeRange._1, timeRange._2 + deltaTimeRange._2) + submitCleanTime(reviseTimeRange._1) + + // read directly through partition info + val partitionRanges = getPartitionRange(reviseTimeRange._1, reviseTimeRange._2) + println(s"read time ranges: ${reviseTimeRange}") + println(s"read partition ranges: ${partitionRanges}") + + // list partition paths + val partitionPaths = listPathsBetweenRanges(filePath :: Nil, partitionRanges) + + if (partitionPaths.isEmpty) { + sqlContext.sparkContext.emptyRDD[Map[String, Any]] + } else { + val filePaths = partitionPaths.mkString(",") + val rdd = sqlContext.sparkContext.textFile(filePaths) + + // decode data + rdd.flatMap { row => + decode(row) + } + } + } + + override def cleanOldData(): Unit = { + val oldCacheLocked = oldCacheLock.lock(-1, TimeUnit.SECONDS) + if (oldCacheLocked) { + try { + val cleanTime = readCleanTime() + cleanTime match { + case Some(ct) => { + // drop partitions + val bounds = getPartition(ct) + + // list partition paths + val earlierPaths = listPathsEarlierThanBounds(filePath :: Nil, bounds) + + // delete out time data path + earlierPaths.foreach { path => + println(s"delete hdfs path: ${path}") + HdfsUtil.deleteHdfsPath(path) + } + } + case _ => { + // do nothing + } + } + } catch { + case e: Throwable => error(s"clean old data error: ${e.getMessage}") + } finally { + oldCacheLock.unlock() + } + } + } + + override def updateOldData(t: Long, oldData: Iterable[Map[String, Any]]): Unit = { + // parallel process different time groups, lock is unnecessary + val ptns = getPartition(t) + val ptnsPath = genPartitionHdfsPath(ptns) + val dirPath = s"${filePath}/${ptnsPath}" + val dataFileName = s"${t}" + val dataFilePath = HdfsUtil.getHdfsFilePath(dirPath, dataFileName) + + try { + // remove out time old data + HdfsFileDumpUtil.remove(dirPath, dataFileName, true) + + // save updated old data + if (oldData.size > 0) { + val recordDatas = oldData.flatMap { dt => + encode(dt, t) + } + val dumped = HdfsFileDumpUtil.dump(dataFilePath, recordDatas, rowSepLiteral) + } + } catch { + case e: Throwable => error(s"update old data error: ${e.getMessage}") + } + } + + override protected def genCleanTime(ms: Long): Long = { + val minPartitionUnit = partitionUnits.last + val t1 = TimeUtil.timeToUnit(ms, minPartitionUnit) + val t2 = TimeUtil.timeFromUnit(t1, minPartitionUnit) + t2 + } + + private def getPartition(ms: Long): List[Long] = { + partitionUnits.map { unit => + TimeUtil.timeToUnit(ms, unit) + } + } + private def getPartitionRange(ms1: Long, ms2: Long): List[(Long, Long)] = { + partitionUnits.map { unit => + val t1 = TimeUtil.timeToUnit(ms1, unit) + val t2 = TimeUtil.timeToUnit(ms2, unit) + (t1, t2) + } + } + + private def genPartitionHdfsPath(partition: List[Long]): String = { + partition.map(prtn => s"${prtn}").mkString("/") + } + + private def str2Long(str: String): Option[Long] = { + try { + Some(str.toLong) + } catch { + case e: Throwable => None + } + } + + // here the range means [min, max], but the best range should be (min, max] + private def listPathsBetweenRanges(paths: List[String], + partitionRanges: List[(Long, Long)] + ): List[String] = { + partitionRanges match { + case Nil => paths + case head :: tail => { + val (lb, ub) = head + val curPaths = paths.flatMap { path => + val names = HdfsUtil.listSubPaths(path, "dir").toList + names.filter { name => + str2Long(name) match { + case Some(t) => (t >= lb) && (t <= ub) + case _ => false + } + }.map(HdfsUtil.getHdfsFilePath(path, _)) + } + listPathsBetweenRanges(curPaths, tail) + } + } + } + + private def listPathsEarlierThanBounds(paths: List[String], bounds: List[Long] + ): List[String] = { + bounds match { + case Nil => paths + case head :: tail => { + val earlierPaths = paths.flatMap { path => + val names = HdfsUtil.listSubPaths(path, "dir").toList + names.filter { name => + str2Long(name) match { + case Some(t) => (t < head) + case _ => false + } + }.map(HdfsUtil.getHdfsFilePath(path, _)) + } + val equalPaths = paths.flatMap { path => + val names = HdfsUtil.listSubPaths(path, "dir").toList + names.filter { name => + str2Long(name) match { + case Some(t) => (t == head) + case _ => false + } + }.map(HdfsUtil.getHdfsFilePath(path, _)) + } + + tail match { + case Nil => earlierPaths + case _ => earlierPaths ::: listPathsEarlierThanBounds(equalPaths, tail) + } + } + } + } + +} diff --git a/measure/src/main/scala/org/apache/griffin/measure/persist/OldHttpPersist.scala b/measure/src/main/scala/org/apache/griffin/measure/persist/OldHttpPersist.scala new file mode 100644 index 000000000..357d6e146 --- /dev/null +++ b/measure/src/main/scala/org/apache/griffin/measure/persist/OldHttpPersist.scala @@ -0,0 +1,87 @@ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ +package org.apache.griffin.measure.persist + +import org.apache.griffin.measure.result._ +import org.apache.griffin.measure.utils.{HttpUtil, JsonUtil} +import org.apache.spark.rdd.RDD + +// persist result by old http way -- temporary way +case class OldHttpPersist(config: Map[String, Any], metricName: String, timeStamp: Long) extends Persist { + + val Api = "api" + val Method = "method" + + val api = config.getOrElse(Api, "").toString + val method = config.getOrElse(Method, "post").toString + + def available(): Boolean = { + api.nonEmpty + } + + def start(msg: String): Unit = {} + def finish(): Unit = {} + + def result(rt: Long, result: Result): Unit = { + result match { + case ar: AccuracyResult => { + val matchPercentage: Double = if (ar.getTotal <= 0) 0 else (ar.getMatch * 1.0 / ar.getTotal) * 100 + val dataMap = Map[String, Any](("metricName" -> metricName), ("timestamp" -> timeStamp), ("value" -> matchPercentage), ("count" -> ar.getTotal)) + httpResult(dataMap) + } + case pr: ProfileResult => { + val dataMap = Map[String, Any](("metricName" -> metricName), ("timestamp" -> timeStamp), ("value" -> pr.getMatch), ("count" -> pr.getTotal)) + httpResult(dataMap) + } + case _ => { + info(s"result: ${result}") + } + } + } + + private def httpResult(dataMap: Map[String, Any]) = { + try { + val data = JsonUtil.toJson(dataMap) + // post + val params = Map[String, Object]() + val header = Map[String, Object](("content-type" -> "application/json")) + + def func(): Boolean = { + HttpUtil.httpRequest(api, method, params, header, data) + } + + PersistThreadPool.addTask(func _, 10) + +// val status = HttpUtil.httpRequest(api, method, params, header, data) +// info(s"${method} to ${api} response status: ${status}") + } catch { + case e: Throwable => error(e.getMessage) + } + + } + + def records(recs: RDD[String], tp: String): Unit = {} + def records(recs: Iterable[String], tp: String): Unit = {} + +// def missRecords(records: RDD[String]): Unit = {} +// def matchRecords(records: RDD[String]): Unit = {} + + def log(rt: Long, msg: String): Unit = {} + +} diff --git a/measure/src/main/scala/org/apache/griffin/measure/persist/PersistFactory.scala b/measure/src/main/scala/org/apache/griffin/measure/persist/PersistFactory.scala index 2532d3ea1..43301608c 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/persist/PersistFactory.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/persist/PersistFactory.scala @@ -27,6 +27,7 @@ case class PersistFactory(persistParams: Iterable[PersistParam], metricName: Str val HDFS_REGEX = """^(?i)hdfs$""".r val HTTP_REGEX = """^(?i)http$""".r + val OLDHTTP_REGEX = """^(?i)oldhttp$""".r val LOG_REGEX = """^(?i)log$""".r def getPersists(timeStamp: Long): MultiPersists = { @@ -39,6 +40,7 @@ case class PersistFactory(persistParams: Iterable[PersistParam], metricName: Str val persistTry = persistParam.persistType match { case HDFS_REGEX() => Try(HdfsPersist(config, metricName, timeStamp)) case HTTP_REGEX() => Try(HttpPersist(config, metricName, timeStamp)) + case OLDHTTP_REGEX() => Try(OldHttpPersist(config, metricName, timeStamp)) case LOG_REGEX() => Try(LoggerPersist(config, metricName, timeStamp)) case _ => throw new Exception("not supported persist type") } diff --git a/measure/src/test/resources/config-streaming3.json b/measure/src/test/resources/config-streaming3.json index 061382b65..fe3e56f70 100644 --- a/measure/src/test/resources/config-streaming3.json +++ b/measure/src/test/resources/config-streaming3.json @@ -19,7 +19,7 @@ "value.type": "java.lang.String" }, "cache": { - "type": "parquet", + "type": "text", "config": { "file.path": "hdfs://localhost/griffin/streaming/dump/source", "info.path": "source", @@ -46,7 +46,7 @@ "value.type": "java.lang.String" }, "cache": { - "type": "parquet", + "type": "text", "config": { "file.path": "hdfs://localhost/griffin/streaming/dump/target", "info.path": "target", diff --git a/measure/src/test/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgoTest.scala b/measure/src/test/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgoTest.scala index be8735a05..529f626a8 100644 --- a/measure/src/test/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgoTest.scala +++ b/measure/src/test/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgoTest.scala @@ -53,7 +53,7 @@ import scala.util.{Failure, Success, Try} class StreamingAccuracyAlgoTest extends FunSuite with Matchers with BeforeAndAfter with Loggable { val envFile = "src/test/resources/env-streaming.json" - val confFile = "src/test/resources/config-streaming2.json" + val confFile = "src/test/resources/config-streaming3.json" val envFsType = "local" val userFsType = "local" @@ -98,8 +98,8 @@ class StreamingAccuracyAlgoTest extends FunSuite with Matchers with BeforeAndAft val conf = new SparkConf().setMaster("local[*]").setAppName(metricName) sc = new SparkContext(conf) sc.setLogLevel(envParam.sparkParam.logLevel) -// sqlContext = new SQLContext(sc) - sqlContext = new HiveContext(sc) + sqlContext = new SQLContext(sc) +// sqlContext = new HiveContext(sc) // val a = sqlContext.sql("select * from s1 limit 10") // // val a = sqlContext.sql("show tables") From 035c4f5585f6e04bf5eaac12feff131e99e22b0f Mon Sep 17 00:00:00 2001 From: Lionel Liu Date: Mon, 7 Aug 2017 16:39:15 +0800 Subject: [PATCH 036/111] v5 --- .../apache/griffin/measure/Application.scala | 19 +++++++++++++------ .../streaming/StreamingAccuracyAlgo.scala | 1 + .../config/params/env/SparkParam.scala | 2 +- measure/src/test/resources/env-streaming.json | 11 ++++++++++- .../streaming/StreamingAccuracyAlgoTest.scala | 2 ++ 5 files changed, 27 insertions(+), 8 deletions(-) diff --git a/measure/src/main/scala/org/apache/griffin/measure/Application.scala b/measure/src/main/scala/org/apache/griffin/measure/Application.scala index 7f385f4cf..af8c83065 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/Application.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/Application.scala @@ -97,16 +97,23 @@ object Application extends Loggable { // algorithm run algo.run match { case Failure(ex) => { - error(ex.getMessage) - sys.exit(-5) + error(s"app error: ${ex.getMessage}") + + procType match { + case ProcessType.streaming() => { + // streaming need to attempt more times by spark streaming itself + throw ex + } + case _ => { + shutdown + sys.exit(-5) + } + } } case _ => { - info("calculation finished") + info("app finished and success") } } - - // shut down - shutdown } private def readParamFile[T <: Param](file: String, fsType: String)(implicit m : Manifest[T]): Try[T] = { diff --git a/measure/src/main/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgo.scala b/measure/src/main/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgo.scala index 1ecba2936..bdac64e0e 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgo.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgo.scala @@ -53,6 +53,7 @@ case class StreamingAccuracyAlgo(allParam: AllParam) extends AccuracyAlgo { val sparkParam = envParam.sparkParam val conf = new SparkConf().setAppName(metricName) + conf.setAll(sparkParam.config) val sc = new SparkContext(conf) sc.setLogLevel(sparkParam.logLevel) val sqlContext = new HiveContext(sc) diff --git a/measure/src/main/scala/org/apache/griffin/measure/config/params/env/SparkParam.scala b/measure/src/main/scala/org/apache/griffin/measure/config/params/env/SparkParam.scala index 872a870ec..6ec095536 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/config/params/env/SparkParam.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/config/params/env/SparkParam.scala @@ -27,7 +27,7 @@ case class SparkParam( @JsonProperty("log.level") logLevel: String, @JsonProperty("checkpoint.dir") cpDir: String, @JsonProperty("batch.interval") batchInterval: String, @JsonProperty("process.interval") processInterval: String, - @JsonProperty("config") config: Map[String, Any] + @JsonProperty("config") config: Map[String, String] ) extends Param { } diff --git a/measure/src/test/resources/env-streaming.json b/measure/src/test/resources/env-streaming.json index c807080b6..42b4aa97d 100644 --- a/measure/src/test/resources/env-streaming.json +++ b/measure/src/test/resources/env-streaming.json @@ -4,7 +4,16 @@ "checkpoint.dir": "hdfs://localhost/test/griffin/cp", "batch.interval": "2s", "process.interval": "10s", - "config": {} + "config": { + "spark.task.maxFailures": 5, + "spark.streaming.kafkaMaxRatePerPartition": 1000, + "spark.streaming.concurrentJobs": 4, + "spark.yarn.maxAppAttempts": 5, + "spark.yarn.am.attemptFailuresValidityInterval": "1h", + "spark.yarn.max.executor.failures": 120, + "spark.yarn.executor.failuresValidityInterval": "1h", + "spark.hadoop.fs.hdfs.impl.disable.cache": true + } }, "persist": [ diff --git a/measure/src/test/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgoTest.scala b/measure/src/test/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgoTest.scala index 529f626a8..cf8b06cc7 100644 --- a/measure/src/test/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgoTest.scala +++ b/measure/src/test/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgoTest.scala @@ -95,7 +95,9 @@ class StreamingAccuracyAlgoTest extends FunSuite with Matchers with BeforeAndAft } val metricName = userParam.name + val sparkParam = envParam.sparkParam val conf = new SparkConf().setMaster("local[*]").setAppName(metricName) + conf.setAll(sparkParam.config) sc = new SparkContext(conf) sc.setLogLevel(envParam.sparkParam.logLevel) sqlContext = new SQLContext(sc) From 8199cb3b3fdce05aeae0a9c0e5487191c2b8c2c7 Mon Sep 17 00:00:00 2001 From: Lionel Liu Date: Mon, 7 Aug 2017 16:56:14 +0800 Subject: [PATCH 037/111] v4 --- .../apache/griffin/measure/algo/batch/BatchAccuracyAlgo.scala | 1 + .../org/apache/griffin/measure/algo/batch/BatchProfileAlgo.scala | 1 + 2 files changed, 2 insertions(+) diff --git a/measure/src/main/scala/org/apache/griffin/measure/algo/batch/BatchAccuracyAlgo.scala b/measure/src/main/scala/org/apache/griffin/measure/algo/batch/BatchAccuracyAlgo.scala index 9240444a2..241f456c0 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/algo/batch/BatchAccuracyAlgo.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/algo/batch/BatchAccuracyAlgo.scala @@ -47,6 +47,7 @@ case class BatchAccuracyAlgo(allParam: AllParam) extends AccuracyAlgo { val sparkParam = envParam.sparkParam val conf = new SparkConf().setAppName(metricName) + conf.setAll(sparkParam.config) val sc = new SparkContext(conf) sc.setLogLevel(sparkParam.logLevel) val sqlContext = new HiveContext(sc) diff --git a/measure/src/main/scala/org/apache/griffin/measure/algo/batch/BatchProfileAlgo.scala b/measure/src/main/scala/org/apache/griffin/measure/algo/batch/BatchProfileAlgo.scala index bc15a508d..163a0b709 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/algo/batch/BatchProfileAlgo.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/algo/batch/BatchProfileAlgo.scala @@ -47,6 +47,7 @@ case class BatchProfileAlgo(allParam: AllParam) extends ProfileAlgo { val sparkParam = envParam.sparkParam val conf = new SparkConf().setAppName(metricName) + conf.setAll(sparkParam.config) val sc = new SparkContext(conf) sc.setLogLevel(sparkParam.logLevel) val sqlContext = new HiveContext(sc) From 6dd674bf8aebd0261b5efa621ad91adf9149a7a0 Mon Sep 17 00:00:00 2001 From: Lionel Liu Date: Wed, 9 Aug 2017 14:54:21 +0800 Subject: [PATCH 038/111] empty --- .../griffin/measure/rule/ExprValueUtil.scala | 14 +++----------- 1 file changed, 3 insertions(+), 11 deletions(-) diff --git a/measure/src/main/scala/org/apache/griffin/measure/rule/ExprValueUtil.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/ExprValueUtil.scala index 83174edad..940d0cb5f 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rule/ExprValueUtil.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rule/ExprValueUtil.scala @@ -234,17 +234,9 @@ object ExprValueUtil { } }) }.toMap - SchemaValueCombineUtil.cartesian(selectionValues) -// val exprValues: Map[String, List[(List[String], Any)]] = -// exprs.foldLeft(Map[String, List[(List[String], Any)]]()) { (existExprValues, expr) => -// existExprValues + (expr._id -> calcExprValues((Nil, data) :: Nil, expr, existExprValueMap).flatMap { pair => -// pair._2 match { -// case Some(v) => Some((pair._1, v)) -// case _ => None -// } -// }) -// } -// SchemaValueCombineUtil.cartesian(exprValues) + // if exprs is empty, return an empty value map for each row + if (selectionValues.isEmpty) List(Map[String, Any]()) + else SchemaValueCombineUtil.cartesian(selectionValues) } // try to calculate some exprs from data and initExprValueMap, generate a new expression value map From 411833ab7b9cc15ada94a5ad00c91677241fb631 Mon Sep 17 00:00:00 2001 From: Lionel Liu Date: Thu, 10 Aug 2017 10:44:12 +0800 Subject: [PATCH 039/111] merge2 --- .../griffin/measure/utils/HdfsUtil.scala | 2 +- .../streaming/StreamingAccuracyAlgoTest.scala | 522 +++++++++--------- .../measure/connector/ConnectorTest.scala | 243 -------- .../measure/connector/test/ParquetTest.scala | 131 ----- 4 files changed, 262 insertions(+), 636 deletions(-) delete mode 100644 measure/src/test/scala/org/apache/griffin/measure/connector/test/ParquetTest.scala diff --git a/measure/src/main/scala/org/apache/griffin/measure/utils/HdfsUtil.scala b/measure/src/main/scala/org/apache/griffin/measure/utils/HdfsUtil.scala index 079359275..6dd54b7f6 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/utils/HdfsUtil.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/utils/HdfsUtil.scala @@ -27,7 +27,7 @@ object HdfsUtil { private val conf = new Configuration() conf.set("dfs.support.append", "true") - conf.set("fs.defaultFS", "hdfs://localhost") // debug @localhost +// conf.set("fs.defaultFS", "hdfs://localhost") // debug @localhost private val dfs = FileSystem.get(conf) diff --git a/measure/src/test/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgoTest.scala b/measure/src/test/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgoTest.scala index cf8b06cc7..a22f91f59 100644 --- a/measure/src/test/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgoTest.scala +++ b/measure/src/test/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgoTest.scala @@ -1,267 +1,267 @@ -/* -Licensed to the Apache Software Foundation (ASF) under one -or more contributor license agreements. See the NOTICE file -distributed with this work for additional information -regarding copyright ownership. The ASF licenses this file -to you under the Apache License, Version 2.0 (the -"License"); you may not use this file except in compliance -with the License. You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, -software distributed under the License is distributed on an -"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -KIND, either express or implied. See the License for the -specific language governing permissions and limitations -under the License. -*/ -package org.apache.griffin.measure.algo.streaming - -import java.util.Date -import java.util.concurrent.TimeUnit - -import org.apache.griffin.measure.algo.batch.BatchAccuracyAlgo -import org.apache.griffin.measure.cache.info.{InfoCacheInstance, TimeInfoCache} -import org.apache.griffin.measure.cache.result._ -import org.apache.griffin.measure.config.params._ -import org.apache.griffin.measure.config.params.env._ -import org.apache.griffin.measure.config.params.user._ -import org.apache.griffin.measure.config.reader._ -import org.apache.griffin.measure.config.validator._ -import org.apache.griffin.measure.connector.direct.DirectDataConnector -import org.apache.griffin.measure.connector.{DataConnector, DataConnectorFactory} -import org.apache.griffin.measure.log.Loggable -import org.apache.griffin.measure.persist.{Persist, PersistFactory, PersistType} -import org.apache.griffin.measure.result._ -import org.apache.griffin.measure.rule.expr._ -import org.apache.griffin.measure.rule.{ExprValueUtil, RuleAnalyzer, RuleFactory} -import org.apache.griffin.measure.utils.{HdfsUtil, TimeUtil} -import org.apache.spark.rdd.RDD -import org.apache.spark.sql.SQLContext -import org.apache.spark.sql.hive.HiveContext -import org.apache.spark.streaming.{Milliseconds, StreamingContext} -import org.apache.spark.{SparkConf, SparkContext} -import org.junit.runner.RunWith -import org.scalatest.junit.JUnitRunner -import org.scalatest.{BeforeAndAfter, FunSuite, Matchers} - -import scala.util.{Failure, Success, Try} - - -@RunWith(classOf[JUnitRunner]) -class StreamingAccuracyAlgoTest extends FunSuite with Matchers with BeforeAndAfter with Loggable { - - val envFile = "src/test/resources/env-streaming.json" - val confFile = "src/test/resources/config-streaming3.json" - val envFsType = "local" - val userFsType = "local" - - val args = Array(envFile, confFile) - - var sc: SparkContext = _ - var sqlContext: SQLContext = _ -// val ssc: StreamingContext = _ - - var allParam: AllParam = _ - - before { - // read param files - val envParam = readParamFile[EnvParam](envFile, envFsType) match { - case Success(p) => p - case Failure(ex) => { - error(ex.getMessage) - sys.exit(-2) - } - } - val userParam = readParamFile[UserParam](confFile, userFsType) match { - case Success(p) => p - case Failure(ex) => { - error(ex.getMessage) - sys.exit(-2) - } - } - allParam = AllParam(envParam, userParam) - - // validate param files - validateParams(allParam) match { - case Failure(ex) => { - error(ex.getMessage) - sys.exit(-3) - } - case _ => { - info("params validation pass") - } - } - - val metricName = userParam.name - val sparkParam = envParam.sparkParam - val conf = new SparkConf().setMaster("local[*]").setAppName(metricName) - conf.setAll(sparkParam.config) - sc = new SparkContext(conf) - sc.setLogLevel(envParam.sparkParam.logLevel) - sqlContext = new SQLContext(sc) -// sqlContext = new HiveContext(sc) - -// val a = sqlContext.sql("select * from s1 limit 10") -// // val a = sqlContext.sql("show tables") -// a.show(10) -// -// val b = HdfsUtil.existPath("/griffin/streaming") -// println(b) - } - - test("algorithm") { - val envParam = allParam.envParam - val userParam = allParam.userParam - val metricName = userParam.name - val sparkParam = envParam.sparkParam - val cleanerParam = envParam.cleanerParam - -// val ssc = StreamingContext.getOrCreate(sparkParam.cpDir, -// ( ) => { -// try { -// val batchInterval = TimeUtil.milliseconds(sparkParam.batchInterval) match { -// case Some(interval) => Milliseconds(interval) -// case _ => throw new Exception("invalid batch interval") -// } -// val ssc = new StreamingContext(sc, batchInterval) -// ssc.checkpoint(sparkParam.cpDir) -// ssc -// } catch { -// case runtime: RuntimeException => { -// throw runtime -// } +///* +//Licensed to the Apache Software Foundation (ASF) under one +//or more contributor license agreements. See the NOTICE file +//distributed with this work for additional information +//regarding copyright ownership. The ASF licenses this file +//to you under the Apache License, Version 2.0 (the +//"License"); you may not use this file except in compliance +//with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +//Unless required by applicable law or agreed to in writing, +//software distributed under the License is distributed on an +//"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +//KIND, either express or implied. See the License for the +//specific language governing permissions and limitations +//under the License. +//*/ +//package org.apache.griffin.measure.algo.streaming +// +//import java.util.Date +//import java.util.concurrent.TimeUnit +// +//import org.apache.griffin.measure.algo.batch.BatchAccuracyAlgo +//import org.apache.griffin.measure.cache.info.{InfoCacheInstance, TimeInfoCache} +//import org.apache.griffin.measure.cache.result._ +//import org.apache.griffin.measure.config.params._ +//import org.apache.griffin.measure.config.params.env._ +//import org.apache.griffin.measure.config.params.user._ +//import org.apache.griffin.measure.config.reader._ +//import org.apache.griffin.measure.config.validator._ +//import org.apache.griffin.measure.connector.direct.DirectDataConnector +//import org.apache.griffin.measure.connector.{DataConnector, DataConnectorFactory} +//import org.apache.griffin.measure.log.Loggable +//import org.apache.griffin.measure.persist.{Persist, PersistFactory, PersistType} +//import org.apache.griffin.measure.result._ +//import org.apache.griffin.measure.rule.expr._ +//import org.apache.griffin.measure.rule.{ExprValueUtil, RuleAnalyzer, RuleFactory} +//import org.apache.griffin.measure.utils.{HdfsUtil, TimeUtil} +//import org.apache.spark.rdd.RDD +//import org.apache.spark.sql.SQLContext +//import org.apache.spark.sql.hive.HiveContext +//import org.apache.spark.streaming.{Milliseconds, StreamingContext} +//import org.apache.spark.{SparkConf, SparkContext} +//import org.junit.runner.RunWith +//import org.scalatest.junit.JUnitRunner +//import org.scalatest.{BeforeAndAfter, FunSuite, Matchers} +// +//import scala.util.{Failure, Success, Try} +// +// +//@RunWith(classOf[JUnitRunner]) +//class StreamingAccuracyAlgoTest extends FunSuite with Matchers with BeforeAndAfter with Loggable { +// +// val envFile = "src/test/resources/env-streaming.json" +// val confFile = "src/test/resources/config-streaming3.json" +// val envFsType = "local" +// val userFsType = "local" +// +// val args = Array(envFile, confFile) +// +// var sc: SparkContext = _ +// var sqlContext: SQLContext = _ +//// val ssc: StreamingContext = _ +// +// var allParam: AllParam = _ +// +// before { +// // read param files +// val envParam = readParamFile[EnvParam](envFile, envFsType) match { +// case Success(p) => p +// case Failure(ex) => { +// error(ex.getMessage) +// sys.exit(-2) +// } +// } +// val userParam = readParamFile[UserParam](confFile, userFsType) match { +// case Success(p) => p +// case Failure(ex) => { +// error(ex.getMessage) +// sys.exit(-2) +// } +// } +// allParam = AllParam(envParam, userParam) +// +// // validate param files +// validateParams(allParam) match { +// case Failure(ex) => { +// error(ex.getMessage) +// sys.exit(-3) +// } +// case _ => { +// info("params validation pass") +// } +// } +// +// val metricName = userParam.name +// val sparkParam = envParam.sparkParam +// val conf = new SparkConf().setMaster("local[*]").setAppName(metricName) +// conf.setAll(sparkParam.config) +// sc = new SparkContext(conf) +// sc.setLogLevel(envParam.sparkParam.logLevel) +// sqlContext = new SQLContext(sc) +//// sqlContext = new HiveContext(sc) +// +//// val a = sqlContext.sql("select * from s1 limit 10") +//// // val a = sqlContext.sql("show tables") +//// a.show(10) +//// +//// val b = HdfsUtil.existPath("/griffin/streaming") +//// println(b) +// } +// +// test("algorithm") { +// val envParam = allParam.envParam +// val userParam = allParam.userParam +// val metricName = userParam.name +// val sparkParam = envParam.sparkParam +// val cleanerParam = envParam.cleanerParam +// +//// val ssc = StreamingContext.getOrCreate(sparkParam.cpDir, +//// ( ) => { +//// try { +//// val batchInterval = TimeUtil.milliseconds(sparkParam.batchInterval) match { +//// case Some(interval) => Milliseconds(interval) +//// case _ => throw new Exception("invalid batch interval") +//// } +//// val ssc = new StreamingContext(sc, batchInterval) +//// ssc.checkpoint(sparkParam.cpDir) +//// ssc +//// } catch { +//// case runtime: RuntimeException => { +//// throw runtime +//// } +//// } +//// }) +// +// val batchInterval = TimeUtil.milliseconds(sparkParam.batchInterval) match { +// case Some(interval) => Milliseconds(interval) +// case _ => throw new Exception("invalid batch interval") +// } +// val ssc = new StreamingContext(sc, batchInterval) +// ssc.checkpoint(sparkParam.cpDir) +// +// // start time +// val startTime = new Date().getTime() +// +// val persistFactory = PersistFactory(envParam.persistParams, metricName) +// +// // get persists to persist measure result +// val appPersist: Persist = persistFactory.getPersists(startTime) +// +// // get spark application id +// val applicationId = sc.applicationId +// +// // persist start id +// appPersist.start(applicationId) +// +// InfoCacheInstance.initInstance(envParam.infoCacheParams, metricName) +// InfoCacheInstance.init +// +// // generate rule from rule param, generate rule analyzer +// val ruleFactory = RuleFactory(userParam.evaluateRuleParam) +// val rule: StatementExpr = ruleFactory.generateRule() +// val ruleAnalyzer: RuleAnalyzer = RuleAnalyzer(rule) +// +// // const expr value map +// val constExprValueMap = ExprValueUtil.genExprValueMaps(None, ruleAnalyzer.constCacheExprs, Map[String, Any]()) +// val finalConstExprValueMap = ExprValueUtil.updateExprValueMaps(ruleAnalyzer.constFinalCacheExprs, constExprValueMap) +// val finalConstMap = finalConstExprValueMap.headOption match { +// case Some(m) => m +// case _ => Map[String, Any]() +// } +// +// // data connector +// val sourceDataConnector: DirectDataConnector = +// DataConnectorFactory.getDirectDataConnector(sqlContext, ssc, userParam.sourceParam, +// ruleAnalyzer.sourceRuleExprs, finalConstMap +// ) match { +// case Success(cntr) => { +// if (cntr.available) cntr +// else throw new Exception("source data connection error!") // } -// }) - - val batchInterval = TimeUtil.milliseconds(sparkParam.batchInterval) match { - case Some(interval) => Milliseconds(interval) - case _ => throw new Exception("invalid batch interval") - } - val ssc = new StreamingContext(sc, batchInterval) - ssc.checkpoint(sparkParam.cpDir) - - // start time - val startTime = new Date().getTime() - - val persistFactory = PersistFactory(envParam.persistParams, metricName) - - // get persists to persist measure result - val appPersist: Persist = persistFactory.getPersists(startTime) - - // get spark application id - val applicationId = sc.applicationId - - // persist start id - appPersist.start(applicationId) - - InfoCacheInstance.initInstance(envParam.infoCacheParams, metricName) - InfoCacheInstance.init - - // generate rule from rule param, generate rule analyzer - val ruleFactory = RuleFactory(userParam.evaluateRuleParam) - val rule: StatementExpr = ruleFactory.generateRule() - val ruleAnalyzer: RuleAnalyzer = RuleAnalyzer(rule) - - // const expr value map - val constExprValueMap = ExprValueUtil.genExprValueMaps(None, ruleAnalyzer.constCacheExprs, Map[String, Any]()) - val finalConstExprValueMap = ExprValueUtil.updateExprValueMaps(ruleAnalyzer.constFinalCacheExprs, constExprValueMap) - val finalConstMap = finalConstExprValueMap.headOption match { - case Some(m) => m - case _ => Map[String, Any]() - } - - // data connector - val sourceDataConnector: DirectDataConnector = - DataConnectorFactory.getDirectDataConnector(sqlContext, ssc, userParam.sourceParam, - ruleAnalyzer.sourceRuleExprs, finalConstMap - ) match { - case Success(cntr) => { - if (cntr.available) cntr - else throw new Exception("source data connection error!") - } - case Failure(ex) => throw ex - } - val targetDataConnector: DirectDataConnector = - DataConnectorFactory.getDirectDataConnector(sqlContext, ssc, userParam.targetParam, - ruleAnalyzer.targetRuleExprs, finalConstMap - ) match { - case Success(cntr) => { - if (cntr.available) cntr - else throw new Exception("target data connection error!") - } - case Failure(ex) => throw ex - } - - val cacheResultProcesser = CacheResultProcesser() - - // init data stream - sourceDataConnector.init() - targetDataConnector.init() - - // my algo - val algo = StreamingAccuracyAlgo(allParam) - - val streamingAccuracyProcess = StreamingAccuracyProcess( - sourceDataConnector, targetDataConnector, - ruleAnalyzer, cacheResultProcesser, persistFactory, appPersist) - - val processInterval = TimeUtil.milliseconds(sparkParam.processInterval) match { - case Some(interval) => interval - case _ => throw new Exception("invalid batch interval") - } - val process = TimingProcess(processInterval, streamingAccuracyProcess) - - // clean thread -// case class Clean() extends Runnable { -// val lock = InfoCacheInstance.genLock("clean") -// def run(): Unit = { -// val locked = lock.lock(5, TimeUnit.SECONDS) -// if (locked) { -// try { -// sourceDataConnector.cleanData -// targetDataConnector.cleanData -// } finally { -// lock.unlock() -// } +// case Failure(ex) => throw ex +// } +// val targetDataConnector: DirectDataConnector = +// DataConnectorFactory.getDirectDataConnector(sqlContext, ssc, userParam.targetParam, +// ruleAnalyzer.targetRuleExprs, finalConstMap +// ) match { +// case Success(cntr) => { +// if (cntr.available) cntr +// else throw new Exception("target data connection error!") // } +// case Failure(ex) => throw ex // } -// } -// val cleanInterval = TimeUtil.milliseconds(cleanerParam.cleanInterval) match { +// +// val cacheResultProcesser = CacheResultProcesser() +// +// // init data stream +// sourceDataConnector.init() +// targetDataConnector.init() +// +// // my algo +// val algo = StreamingAccuracyAlgo(allParam) +// +// val streamingAccuracyProcess = StreamingAccuracyProcess( +// sourceDataConnector, targetDataConnector, +// ruleAnalyzer, cacheResultProcesser, persistFactory, appPersist) +// +// val processInterval = TimeUtil.milliseconds(sparkParam.processInterval) match { // case Some(interval) => interval // case _ => throw new Exception("invalid batch interval") // } -// val clean = TimingProcess(cleanInterval, Clean()) - - process.startup() -// clean.startup() - - ssc.start() - ssc.awaitTermination() - ssc.stop(stopSparkContext=true, stopGracefully=true) - - println("================ end ================") - - // context stop - sc.stop - - InfoCacheInstance.close - - appPersist.finish() - - process.shutdown() -// clean.shutdown() - } - - private def readParamFile[T <: Param](file: String, fsType: String)(implicit m : Manifest[T]): Try[T] = { - val paramReader = ParamReaderFactory.getParamReader(file, fsType) - paramReader.readConfig[T] - } - - private def validateParams(allParam: AllParam): Try[Boolean] = { - val allParamValidator = AllParamValidator() - allParamValidator.validate(allParam) - } - -} +// val process = TimingProcess(processInterval, streamingAccuracyProcess) +// +// // clean thread +//// case class Clean() extends Runnable { +//// val lock = InfoCacheInstance.genLock("clean") +//// def run(): Unit = { +//// val locked = lock.lock(5, TimeUnit.SECONDS) +//// if (locked) { +//// try { +//// sourceDataConnector.cleanData +//// targetDataConnector.cleanData +//// } finally { +//// lock.unlock() +//// } +//// } +//// } +//// } +//// val cleanInterval = TimeUtil.milliseconds(cleanerParam.cleanInterval) match { +//// case Some(interval) => interval +//// case _ => throw new Exception("invalid batch interval") +//// } +//// val clean = TimingProcess(cleanInterval, Clean()) +// +// process.startup() +//// clean.startup() +// +// ssc.start() +// ssc.awaitTermination() +// ssc.stop(stopSparkContext=true, stopGracefully=true) +// +// println("================ end ================") +// +// // context stop +// sc.stop +// +// InfoCacheInstance.close +// +// appPersist.finish() +// +// process.shutdown() +//// clean.shutdown() +// } +// +// private def readParamFile[T <: Param](file: String, fsType: String)(implicit m : Manifest[T]): Try[T] = { +// val paramReader = ParamReaderFactory.getParamReader(file, fsType) +// paramReader.readConfig[T] +// } +// +// private def validateParams(allParam: AllParam): Try[Boolean] = { +// val allParamValidator = AllParamValidator() +// allParamValidator.validate(allParam) +// } +// +//} diff --git a/measure/src/test/scala/org/apache/griffin/measure/connector/ConnectorTest.scala b/measure/src/test/scala/org/apache/griffin/measure/connector/ConnectorTest.scala index 9cb6dfab3..2139ff77b 100644 --- a/measure/src/test/scala/org/apache/griffin/measure/connector/ConnectorTest.scala +++ b/measure/src/test/scala/org/apache/griffin/measure/connector/ConnectorTest.scala @@ -66,248 +66,5 @@ class ConnectorTest extends FunSuite with Matchers with BeforeAndAfter { val clazz = Class.forName(tp) ClassTag(clazz) } - -// private def getDeserializer(ct: ClassTag[_]): String = { -// ct.runtimeClass.get -// ct match { -// case Some(t: scala.Predef.Class[String]) => "kafka.serializer.StringDecoder" -// } -// } - -// "config": { -// "kafka.config": { -// "bootstrap.servers": "localhost:9092", -// "group.id": "group1", -// "auto.offset.reset": "smallest", -// "auto.commit.enable": "false", -// }, -// "topics": "sss", -// "key.type": "java.lang.String", -// "value.type": "java.lang.String", -// "cache": { -// "type": "temp", -// "config": { -// "table.name": "source", -// "info.path": "src" -// } -// } -// } - -// test("connector") { - // val kafkaConfig = Map[String, String]( - // ("bootstrap.servers" -> "10.149.247.156:9092"), - // ("group.id" -> "test"), - // ("auto.offset.reset" -> "smallest"), - // ("auto.commit.enable" -> "false") - // ) - // - // val cacheConfig = Map[String, Any]( - // ("table.name" -> "source"), - // ("info.path" -> "src") - // ) - // - // val cacheParam = DataCacheParam("df", cacheConfig, Nil) - // - // val config = Map[String, Any]( - // ("kafka.config" -> kafkaConfig), - // ("topics" -> "sss"), - // ("key.type" -> "java.lang.String"), - // ("value.type" -> "java.lang.String") - // ) - // - // val infoCacheConfig = Map[String, Any]( - // ("hosts" -> "localhost:2181"), - // ("namespace" -> "griffin/infocache"), - // ("lock.path" -> "lock"), - // ("mode" -> "persist"), - // ("init.clear" -> true), - // ("close.clear" -> false) - // ) - // val name = "ttt" - // - // val icp = InfoCacheParam("zk", infoCacheConfig) - // val icps = icp :: Nil - // - // InfoCacheInstance.initInstance(icps, name) - // InfoCacheInstance.init - // - // - // val connectorParam = DataConnectorParam("kafka", "0.8", config, cacheParam, false) - // - // val conf = new SparkConf().setMaster("local[*]").setAppName("ConnectorTest") - // val sc = new SparkContext(conf) - // sc.setLogLevel("WARN") - // val sqlContext = new SQLContext(sc) - // - // val batchInterval = TimeUtil.milliseconds("2s") match { - // case Some(interval) => Milliseconds(interval) - // case _ => throw new Exception("invalid batch interval") - // } - // val ssc = new StreamingContext(sc, batchInterval) - // ssc.checkpoint("/test/griffin/cp") - // - // val connector = DataConnectorFactory.getStreamingDataConnector(ssc, connectorParam) - // - // val streamingConnector = connector match { - // case Success(c) => c - // case _ => fail - // } - // - // val dataCacheParam = connectorParam.cache - // val cacheDataConnector = DataConnectorFactory.getCacheDataConnector(sqlContext, dataCacheParam) match { - // case Success(cntr) => cntr - // case Failure(ex) => throw ex - // } - // - // /// - // - //// def genDataFrame(rdd: RDD[Map[String, Any]]): DataFrame = { - //// val fields = rdd.aggregate(Map[String, DataType]())( - //// DataTypeCalculationUtil.sequenceDataTypeMap, DataTypeCalculationUtil.combineDataTypeMap - //// ).toList.map(f => StructField(f._1, f._2)) - //// val schema = StructType(fields) - //// val datas: RDD[Row] = rdd.map { d => - //// val values = fields.map { field => - //// val StructField(k, dt, _, _) = field - //// d.get(k) match { - //// case Some(v) => v - //// case _ => null - //// } - //// } - //// Row(values: _*) - //// } - //// val df = sqlContext.createDataFrame(datas, schema) - //// df - //// } - // - // val rules = "$source.json().name = 's2' AND $source.json().age = 32" - // val ep = EvaluateRuleParam(1, rules) - // - // val ruleFactory = RuleFactory(ep) - // val rule: StatementExpr = ruleFactory.generateRule() - // val ruleAnalyzer: RuleAnalyzer = RuleAnalyzer(rule) - // - // val ruleExprs = ruleAnalyzer.sourceRuleExprs - // val constFinalExprValueMap = Map[String, Any]() - // - // /// - // - // val ds = streamingConnector.stream match { - // case Success(dstream) => dstream - // case Failure(ex) => throw ex - // } - // - // ds.foreachRDD((rdd, time) => { - // val ms = time.milliseconds - // - // val data = rdd.collect - // val str = data.mkString("\n") - // - // println(s"${ms}: \n${str}") - // - // val dataInfoMap = DataInfo.cacheInfoList.map(_.defWrap).toMap + TimeStampInfo.wrap(ms) - // - // // parse each message - // val valueMapRdd: RDD[Map[String, Any]] = rdd.flatMap { kv => - // val msg = kv._2 - // - // val cacheExprValueMaps = ExprValueUtil.genExprValueMaps(Some(msg), ruleExprs.cacheExprs, constFinalExprValueMap) - // val finalExprValueMaps = ExprValueUtil.updateExprValueMaps(ruleExprs.finalCacheExprs, cacheExprValueMaps) - // - // // val sf = StructField("name", DataType.fromJson("string")) - // // val schema: StructType = new StructType() - // - // finalExprValueMaps.map { vm => - // vm ++ dataInfoMap - // } - // } - // - // val cnt = valueMapRdd.count - // - // val valueMaps = valueMapRdd.collect() - // val valuestr = valueMaps.mkString("\n") - // - //// println(s"count: ${cnt}\n${valuestr}") - // - // // generate DataFrame - //// val df = genDataFrame(valueMapRdd) - //// df.show(10) - // - // // save data frame - // cacheDataConnector.saveData(valueMapRdd, ms) - // - // // show data - //// cacheDataConnector.readData() match { - //// case Success(rdf) => rdf.show(10) - //// case Failure(ex) => println(s"cache data error: ${ex.getMessage}") - //// } - //// - //// cacheDataConnector.submitLastProcTime(ms) - // }) - // - // // process thread - // case class Process() extends Runnable { - // val lock = InfoCacheInstance.genLock("process") - // def run(): Unit = { - // val locked = lock.lock(5, TimeUnit.SECONDS) - // if (locked) { - // try { - // // show data - // cacheDataConnector.readData() match { - // case Success(rdd) => { - // rdd.take(10).foreach(println) - // println(s"count: ${rdd.count}") - // } - // case Failure(ex) => println(s"cache data error: ${ex.getMessage}") - // } - // - //// val st = new Date().getTime - // // get data - //// val sourceData = sourceDataConnector.data match { - //// case Success(dt) => dt - //// case Failure(ex) => throw ex - //// } - //// val targetData = targetDataConnector.data match { - //// case Success(dt) => dt - //// case Failure(ex) => throw ex - //// } - //// - //// // accuracy algorithm - //// val (accuResult, missingRdd, matchedRdd) = accuracy(sourceData, targetData, ruleAnalyzer) - //// - //// println(accuResult) - //// - //// val et = new Date().getTime - //// - //// val missingRecords = missingRdd.map(record2String(_, ruleAnalyzer.sourceRuleExprs.persistExprs, ruleAnalyzer.targetRuleExprs.persistExprs)) - // - // } finally { - // lock.unlock() - // } - // } - // } - // } - // - // val processInterval = TimeUtil.milliseconds("10s") match { - // case Some(interval) => interval - // case _ => throw new Exception("invalid batch interval") - // } - // val process = TimingProcess(processInterval, Process()) - // - // process.startup() - // - // - // ssc.start() - // ssc.awaitTermination() - // ssc.stop(stopSparkContext=true, stopGracefully=true) - // - // // context stop - // sc.stop - // - // InfoCacheInstance.close() - // - // process.shutdown() - // - // } } diff --git a/measure/src/test/scala/org/apache/griffin/measure/connector/test/ParquetTest.scala b/measure/src/test/scala/org/apache/griffin/measure/connector/test/ParquetTest.scala deleted file mode 100644 index 5bdc17556..000000000 --- a/measure/src/test/scala/org/apache/griffin/measure/connector/test/ParquetTest.scala +++ /dev/null @@ -1,131 +0,0 @@ -package org.apache.griffin.measure.connector.test - - -import java.util.Date - -import org.apache.griffin.measure.rule.DataTypeCalculationUtil -import org.apache.griffin.measure.utils.HdfsUtil -import org.apache.spark.rdd.RDD -import org.apache.spark.sql.{DataFrame, Row, SQLContext} -import org.apache.spark.sql.hive.HiveContext -import org.apache.spark.sql.types.{DataType, StructField, StructType} -import org.apache.spark.{SparkConf, SparkContext} -import org.junit.runner.RunWith -import org.scalatest.junit.JUnitRunner -import org.scalatest.{BeforeAndAfter, FunSuite, Matchers} - - -@RunWith(classOf[JUnitRunner]) -class ParquetTest extends FunSuite with Matchers with BeforeAndAfter { - - test ("parquet") { - val conf = new SparkConf().setMaster("local[*]").setAppName("parquet") - val sc = new SparkContext(conf) -// sc.setLogLevel(envParam.sparkParam.logLevel) - val sqlContext = new SQLContext(sc) -// sqlContext = new HiveContext(sc) - - val t1 = new Date().getTime() - -// val data = (0 to 99999).toList.map(d => Map[String, Any]( -// ("name" -> s"s${d}"), -// ("age" -> (d % 10)) -// )) - - case class Data(name: String, age: Long) { - def getName = name - def getAge = age - } - val data = (0 to 99999).toList.map(d => Data(s"s${d}", d % 10)) - - val rdd = sc.parallelize(data) - -// val df = genDataFrame(sqlContext, rdd) - val df = sqlContext.createDataFrame(rdd, classOf[Data]) - println(df.count) - - val t2 = new Date().getTime() - - df.write.partitionBy("age").parquet("hdfs://localhost/test/parq") - - val t3 = new Date().getTime() - println(s"write time: ${t3 - t2}") - - val readDf = sqlContext.read.parquet("hdfs://localhost/test/parq") - readDf.show() - println(readDf.count) - - val t4 = new Date().getTime() - println(s"read time: ${t4 - t3}") - - } - - private def genDataFrame(sqlContext: SQLContext, rdd: RDD[Map[String, Any]]): DataFrame = { - val fields = rdd.aggregate(Map[String, DataType]())( - DataTypeCalculationUtil.sequenceDataTypeMap, DataTypeCalculationUtil.combineDataTypeMap - ).toList.map(f => StructField(f._1, f._2)) - val schema = StructType(fields) - val datas: RDD[Row] = rdd.map { d => - val values = fields.map { field => - val StructField(k, dt, _, _) = field - d.get(k) match { - case Some(v) => v - case _ => null - } - } - Row(values: _*) - } - val df = sqlContext.createDataFrame(datas, schema) - df - } - - test ("list") { - val conf = new SparkConf().setMaster("local[*]").setAppName("parquet") - val sc = new SparkContext(conf) - // sc.setLogLevel(envParam.sparkParam.logLevel) - val sqlContext = new SQLContext(sc) - - val filePath = "hdfs://localhost/griffin/streaming/dump/source" - - val partitionRanges: List[(Long, Long)] = List((417007L,417007L), (25020456L,25020459L)) - - println(partitionRanges) - - // list partition paths - val partitionPaths = listPartitionPathsByRanges(filePath :: Nil, partitionRanges) - - partitionPaths.foreach(println) - } - - private def listPartitionPathsByRanges(paths: List[String], partitionRanges: List[(Long, Long)] - ): List[String] = { - partitionRanges match { - case Nil => paths - case head :: tail => { - val curPaths = listPartitionPathsByRange(paths, head) - listPartitionPathsByRanges(curPaths, tail) - } - } - } - - private def listPartitionPathsByRange(paths: List[String], partitionRange: (Long, Long) - ): List[String] = { - val (lb, ub) = partitionRange - paths.flatMap { path => - val names = HdfsUtil.listSubPaths(path, "dir").toList - names.filter { name => - val t = str2Long(name) - (t > 0) && (t >= lb) && (t <= ub) - }.map(HdfsUtil.getHdfsFilePath(path, _)) - } - } - - private def str2Long(str: String): Long = { - try { - str.toLong - } catch { - case e: Throwable => -1 - } - } - -} From 7b60c1097a6673a19c98397409b908e3aa5aee1c Mon Sep 17 00:00:00 2001 From: Lionel Liu Date: Thu, 10 Aug 2017 11:11:33 +0800 Subject: [PATCH 040/111] remove df and parquet --- .../streaming/StreamingAccuracyProcess.scala | 10 +- .../connector/DataConnectorFactory.scala | 4 - .../cache/DfCacheDataConnector.scala | 281 ------------- .../cache/ParquetCacheDataConnector.scala | 370 ------------------ 4 files changed, 1 insertion(+), 664 deletions(-) delete mode 100644 measure/src/main/scala/org/apache/griffin/measure/connector/cache/DfCacheDataConnector.scala delete mode 100644 measure/src/main/scala/org/apache/griffin/measure/connector/cache/ParquetCacheDataConnector.scala diff --git a/measure/src/main/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyProcess.scala b/measure/src/main/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyProcess.scala index 7f1e63e2e..be1f846d6 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyProcess.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyProcess.scala @@ -143,19 +143,11 @@ case class StreamingAccuracyProcess(sourceDataConnector: DirectDataConnector, v ++ i } sourceDataConnector.updateOldData(t, dumpDatas) - // targetDataConnector.updateOldData(t, dumpDatas) // not correct +// targetDataConnector.updateOldData(t, dumpDatas) // not correct } updateResults.unpersist() - // dump missing rdd (this part not need for future version, only for current df cache data version) - val dumpRdd: RDD[Map[String, Any]] = missingRdd.map { r => - val (_, (v, i)) = r - v ++ i - } - sourceDataConnector.updateAllOldData(dumpRdd) - targetDataConnector.updateAllOldData(dumpRdd) // not correct - TimeInfoCache.endTimeInfoCache // clean old data diff --git a/measure/src/main/scala/org/apache/griffin/measure/connector/DataConnectorFactory.scala b/measure/src/main/scala/org/apache/griffin/measure/connector/DataConnectorFactory.scala index 517695ced..670175d2f 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/connector/DataConnectorFactory.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/connector/DataConnectorFactory.scala @@ -39,8 +39,6 @@ object DataConnectorFactory { val KafkaRegex = """^(?i)kafka$""".r - val DfRegex = """^(?i)df|dataframe$""".r - val ParquetRegex = """^(?i)parquet$""".r val TextRegex = """^(?i)text$""".r def getDirectDataConnector(sqlContext: SQLContext, @@ -91,9 +89,7 @@ object DataConnectorFactory { val cacheType = dataCacheParam.cacheType Try { cacheType match { - case DfRegex() => DfCacheDataConnector(sqlContext, dataCacheParam) case HiveRegex() => HiveCacheDataConnector(sqlContext, dataCacheParam) - case ParquetRegex() => ParquetCacheDataConnector(sqlContext, dataCacheParam) case TextRegex() => TextCacheDataConnector(sqlContext, dataCacheParam) case _ => throw new Exception("cache connector creation error!") } diff --git a/measure/src/main/scala/org/apache/griffin/measure/connector/cache/DfCacheDataConnector.scala b/measure/src/main/scala/org/apache/griffin/measure/connector/cache/DfCacheDataConnector.scala deleted file mode 100644 index f883b4fec..000000000 --- a/measure/src/main/scala/org/apache/griffin/measure/connector/cache/DfCacheDataConnector.scala +++ /dev/null @@ -1,281 +0,0 @@ -/* -Licensed to the Apache Software Foundation (ASF) under one -or more contributor license agreements. See the NOTICE file -distributed with this work for additional information -regarding copyright ownership. The ASF licenses this file -to you under the Apache License, Version 2.0 (the -"License"); you may not use this file except in compliance -with the License. You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, -software distributed under the License is distributed on an -"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -KIND, either express or implied. See the License for the -specific language governing permissions and limitations -under the License. -*/ -package org.apache.griffin.measure.connector.cache - -import java.util.concurrent.TimeUnit - -import org.apache.griffin.measure.cache.info.{InfoCacheInstance, TimeInfoCache} -import org.apache.griffin.measure.config.params.user.DataCacheParam -import org.apache.griffin.measure.result.TimeStampInfo -import org.apache.griffin.measure.rule.DataTypeCalculationUtil -import org.apache.griffin.measure.utils.TimeUtil -import org.apache.spark.rdd.RDD -import org.apache.spark.sql.types._ -import org.apache.spark.sql.{DataFrame, Row, SQLContext} -import org.apache.spark.storage.StorageLevel - -import scala.util.Try - -case class DfCacheDataConnector(sqlContext: SQLContext, dataCacheParam: DataCacheParam - ) extends CacheDataConnector { - - val config = dataCacheParam.config - val InfoPath = "info.path" - val cacheInfoPath: String = config.getOrElse(InfoPath, defCacheInfoPath).toString - - val newCacheLock = InfoCacheInstance.genLock(s"${cacheInfoPath}.new") - val oldCacheLock = InfoCacheInstance.genLock(s"${cacheInfoPath}.old") - - val timeRangeParam: List[String] = if (dataCacheParam.timeRange != null) dataCacheParam.timeRange else Nil - val deltaTimeRange: (Long, Long) = (timeRangeParam ::: List("0", "0")) match { - case s :: e :: _ => { - val ns = TimeUtil.milliseconds(s) match { - case Some(n) if (n < 0) => n - case _ => 0 - } - val ne = TimeUtil.milliseconds(e) match { - case Some(n) if (n < 0) => n - case _ => 0 - } - (ns, ne) - } - case _ => (0, 0) - } - - val CacheLevel = "cache.level" - val cacheLevel: String = config.getOrElse(CacheLevel, "MEMORY_AND_DISK").toString - - val timeStampColumn = TimeStampInfo.key - - var newDataFrame: DataFrame = null - var oldDataFrame: DataFrame = null - - val ReadyTimeInterval = "ready.time.interval" - val ReadyTimeDelay = "ready.time.delay" - val readyTimeInterval: Long = TimeUtil.milliseconds(config.getOrElse(ReadyTimeInterval, "1m").toString).getOrElse(60000L) - val readyTimeDelay: Long = TimeUtil.milliseconds(config.getOrElse(ReadyTimeDelay, "1m").toString).getOrElse(60000L) - - def available(): Boolean = { - true - } - - def init(): Unit = {} - - def saveData(rdd: RDD[Map[String, Any]], ms: Long): Unit = { - val newCacheLocked = newCacheLock.lock(-1, TimeUnit.SECONDS) - if (newCacheLocked) { - try { - if (newDataFrame == null) { - if (!rdd.isEmpty) { - newDataFrame = genDataFrame(rdd) - newDataFrame.persist(StorageLevel.fromString(cacheLevel)) - } - } else { - if (!rdd.isEmpty) { - newDataFrame.unpersist() - newDataFrame = newDataFrame.unionAll(genDataFrame(rdd)) - newDataFrame.persist(StorageLevel.fromString(cacheLevel)) - } - } - // submit ms - submitCacheTime(ms) - submitReadyTime(ms) - } catch { - case e: Throwable => error(s"save data error: ${e.getMessage}") - } finally { - newCacheLock.unlock() - } - } - } - - def readData(): Try[RDD[Map[String, Any]]] = Try { - val timeRange = TimeInfoCache.getTimeRange - submitLastProcTime(timeRange._2) - val reviseTimeRange = (timeRange._1 + deltaTimeRange._1, timeRange._2 + deltaTimeRange._2) - - // move new data frame to temp data frame - val newCacheLocked = newCacheLock.lock(-1, TimeUnit.SECONDS) - val newTempDataFrame = if (newCacheLocked) { - try { - val tmp = newDataFrame.filter(s"${timeStampColumn} BETWEEN ${reviseTimeRange._1} AND ${reviseTimeRange._2}") - newDataFrame.unpersist() - newDataFrame = newDataFrame.filter(s"${timeStampColumn} > ${reviseTimeRange._2}") - tmp - } catch { - case _ => null - } finally { - newCacheLock.unlock() - } - } else null - - // add temp data frame to old data frame - val oldCacheLocked = oldCacheLock.lock(-1, TimeUnit.SECONDS) - val oldTempDataFrame = if (oldCacheLocked) { - try { - if (oldDataFrame != null) { - oldDataFrame.filter(s"${timeStampColumn} BETWEEN ${reviseTimeRange._1} AND ${reviseTimeRange._2}") - } else null - } catch { - case _ => null - } finally { - oldCacheLock.unlock() - } - } else null - - val resultDataFrame = if (oldTempDataFrame == null && newTempDataFrame == null) { - throw new Exception("data not cached") - } else { - val finalDataFrame = if (newTempDataFrame == null) { - oldTempDataFrame - } else if (oldTempDataFrame == null) { - newTempDataFrame - } else { - oldTempDataFrame.unionAll(newTempDataFrame) - } - finalDataFrame - } - - // data frame -> rdd - resultDataFrame.map { row => - SparkRowFormatter.formatRow(row) - } - } - - override def cleanOldData(): Unit = { - val oldCacheLocked = oldCacheLock.lock(-1, TimeUnit.SECONDS) - if (oldCacheLocked) { - try { - val timeRange = TimeInfoCache.getTimeRange - val reviseTimeRange = (timeRange._1 + deltaTimeRange._1, timeRange._2 + deltaTimeRange._2) - - oldDataFrame.unpersist() - oldDataFrame = oldDataFrame.filter(s"${timeStampColumn} >= ${reviseTimeRange._1}") - oldDataFrame.persist(StorageLevel.fromString(cacheLevel)) - } catch { - case e: Throwable => error(s"clean old data error: ${e.getMessage}") - } finally { - oldCacheLock.unlock() - } - } - -// if (initialed) { -// val timeRange = TimeInfoCache.getTimeRange -// val reviseTimeRange = (timeRange._1 + deltaTimeRange._1, timeRange._2 + deltaTimeRange._2) -// println(s"clean reviseTimeRange: ${reviseTimeRange}") -// -// dataFrame.show(10) -// -// dataFrame.unpersist() -// dataFrame = dataFrame.filter(s"${timeStampColumn} >= ${reviseTimeRange._1}") -// dataFrame.persist(StorageLevel.fromString(cacheLevel)) -// -// dataFrame.show(10) -// } - } - - override def updateAllOldData(oldRdd: RDD[Map[String, Any]]): Unit = { - val oldCacheLocked = oldCacheLock.lock(-1, TimeUnit.SECONDS) - if (oldCacheLocked) { - try { - if (oldDataFrame == null) { - if (!oldRdd.isEmpty) { - oldDataFrame = genDataFrame(oldRdd) - oldDataFrame.persist(StorageLevel.fromString(cacheLevel)) - } - } else { - if (!oldRdd.isEmpty) { - oldDataFrame.unpersist() - oldDataFrame = genDataFrame(oldRdd) - oldDataFrame.persist(StorageLevel.fromString(cacheLevel)) - } else { - oldDataFrame.unpersist() - oldDataFrame = null - } - } - } catch { - case e: Throwable => error(s"update all old data error: ${e.getMessage}") - } finally { - oldCacheLock.unlock() - } - } - } - - // generate DataFrame - // maybe we can directly use def createDataFrame[A <: Product : TypeTag](rdd: RDD[A]): DataFrame - // to avoid generate data type by myself, just translate each value into Product - private def genDataFrame(rdd: RDD[Map[String, Any]]): DataFrame = { - val fields = rdd.aggregate(Map[String, DataType]())( - DataTypeCalculationUtil.sequenceDataTypeMap, DataTypeCalculationUtil.combineDataTypeMap - ).toList.map(f => StructField(f._1, f._2)) - val schema = StructType(fields) - val datas: RDD[Row] = rdd.map { d => - val values = fields.map { field => - val StructField(k, dt, _, _) = field - d.get(k) match { - case Some(v) => v - case _ => null - } - } - Row(values: _*) - } - val df = sqlContext.createDataFrame(datas, schema) - df - } - -} - -import scala.collection.mutable.ArrayBuffer - -object SparkRowFormatter { - - def formatRow(row: Row): Map[String, Any] = { - formatRowWithSchema(row, row.schema) - } - - private def formatRowWithSchema(row: Row, schema: StructType): Map[String, Any] = { - formatStruct(schema.fields, row) - } - - private def formatStruct(schema: Seq[StructField], r: Row) = { - val paired = schema.zip(r.toSeq) - paired.foldLeft(Map[String, Any]())((s, p) => s ++ formatItem(p)) - } - - private def formatItem(p: Pair[StructField, Any]): Map[String, Any] = { - p match { - case (sf, a) => - sf.dataType match { - case ArrayType(et, _) => - Map(sf.name -> (if (a == null) a else formatArray(et, a.asInstanceOf[ArrayBuffer[Any]]))) - case StructType(s) => - Map(sf.name -> (if (a == null) a else formatStruct(s, a.asInstanceOf[Row]))) - case _ => Map(sf.name -> a) - } - } - } - - private def formatArray(et: DataType, arr: ArrayBuffer[Any]): Seq[Any] = { - et match { - case StructType(s) => arr.map(e => formatStruct(s, e.asInstanceOf[Row])) - case ArrayType(t, _) => - arr.map(e => formatArray(t, e.asInstanceOf[ArrayBuffer[Any]])) - case _ => arr - } - } -} \ No newline at end of file diff --git a/measure/src/main/scala/org/apache/griffin/measure/connector/cache/ParquetCacheDataConnector.scala b/measure/src/main/scala/org/apache/griffin/measure/connector/cache/ParquetCacheDataConnector.scala deleted file mode 100644 index b641d22fe..000000000 --- a/measure/src/main/scala/org/apache/griffin/measure/connector/cache/ParquetCacheDataConnector.scala +++ /dev/null @@ -1,370 +0,0 @@ -/* -Licensed to the Apache Software Foundation (ASF) under one -or more contributor license agreements. See the NOTICE file -distributed with this work for additional information -regarding copyright ownership. The ASF licenses this file -to you under the Apache License, Version 2.0 (the -"License"); you may not use this file except in compliance -with the License. You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, -software distributed under the License is distributed on an -"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -KIND, either express or implied. See the License for the -specific language governing permissions and limitations -under the License. -*/ -package org.apache.griffin.measure.connector.cache - -import java.util.concurrent.TimeUnit - -import org.apache.griffin.measure.cache.info.{InfoCacheInstance, TimeInfoCache} -import org.apache.griffin.measure.config.params.user.DataCacheParam -import org.apache.griffin.measure.result.TimeStampInfo -import org.apache.griffin.measure.utils.{HdfsUtil, JsonUtil, TimeUtil} -import org.apache.spark.rdd.RDD -import org.apache.spark.sql.{Row, SQLContext} -import org.apache.spark.sql.types._ -import scala.collection.JavaConversions._ - -import scala.util.Try - -case class ParquetCacheDataConnector(sqlContext: SQLContext, dataCacheParam: DataCacheParam - ) extends CacheDataConnector { - - val config = dataCacheParam.config - val InfoPath = "info.path" - val cacheInfoPath: String = config.getOrElse(InfoPath, defCacheInfoPath).toString - - val newCacheLock = InfoCacheInstance.genLock(s"${cacheInfoPath}.new") - val oldCacheLock = InfoCacheInstance.genLock(s"${cacheInfoPath}.old") - - val timeRangeParam: List[String] = if (dataCacheParam.timeRange != null) dataCacheParam.timeRange else Nil - val deltaTimeRange: (Long, Long) = (timeRangeParam ::: List("0", "0")) match { - case s :: e :: _ => { - val ns = TimeUtil.milliseconds(s) match { - case Some(n) if (n < 0) => n - case _ => 0 - } - val ne = TimeUtil.milliseconds(e) match { - case Some(n) if (n < 0) => n - case _ => 0 - } - (ns, ne) - } - case _ => (0, 0) - } - - val FilePath = "file.path" - val filePath: String = config.get(FilePath) match { - case Some(s: String) => s - case _ => throw new Exception("invalid file.path!") - } - - val ReadyTimeInterval = "ready.time.interval" - val ReadyTimeDelay = "ready.time.delay" - val readyTimeInterval: Long = TimeUtil.milliseconds(config.getOrElse(ReadyTimeInterval, "1m").toString).getOrElse(60000L) - val readyTimeDelay: Long = TimeUtil.milliseconds(config.getOrElse(ReadyTimeDelay, "1m").toString).getOrElse(60000L) - - val TimeStampColumn: String = TimeStampInfo.key - val PayloadColumn: String = "payload" - - // cache schema: Long, String - val fields = List[StructField]( - StructField(TimeStampColumn, LongType), - StructField(PayloadColumn, StringType) - ) - val schema = StructType(fields) - -// case class CacheData(time: Long, payload: String) { -// def getTime(): Long = time -// def getPayload(): String = payload -// } - - val partitionUnits: List[String] = List("hour", "min") - - override def init(): Unit = { - // do nothing - } - - def available(): Boolean = { - true - } - - private def encode(data: Map[String, Any], ms: Long): Option[Row] = { - try { - val values = fields.map { field => - val StructField(name, _, _, _) = field - name match { - case TimeStampColumn => ms - case PayloadColumn => JsonUtil.toJson(data) - case _ => null - } - } - Some(Row(values: _*)) - } catch { - case _ => None - } - } - - private def decode(data: Row, updateTimeStamp: Boolean): Option[Map[String, Any]] = { - data.getAs[Any](PayloadColumn) match { - case v: String => { - try { - val map = JsonUtil.toAnyMap(v) - val resMap = if (updateTimeStamp) { - data.getAs[Any](TimeStampColumn) match { - case t: Long => map + (TimeStampColumn -> t) - case _ => map - } - } else map - Some(resMap) - } catch { - case _ => None - } - } - case _ => None - } - } - -// private def encode(data: Map[String, Any], ms: Long): Option[CacheData] = { -// try { -// val json = JsonUtil.toJson(data) -// Some(CacheData(ms, json)) -// } catch { -// case _ => None -// } -// } -// -// private def decode(data: Row, updateTimeStamp: Boolean): Option[Map[String, Any]] = { -// data.getAs[Any]("payload") match { -// case v: String => { -// try { -// val map = JsonUtil.toAnyMap(v) -// val resMap = if (updateTimeStamp) { -// data.getAs[Any]("time") match { -// case t: Long => map + (TimeStampInfo.key -> t) -// case _ => map -// } -// } else map -// Some(resMap) -// } catch { -// case _ => None -// } -// } -// case _ => None -// } -// } - - def saveData(rdd: RDD[Map[String, Any]], ms: Long): Unit = { - val newCacheLocked = newCacheLock.lock(-1, TimeUnit.SECONDS) - if (newCacheLocked) { - try { - if (!rdd.isEmpty) { - val ptns = getPartition(ms) - val ptnsPath = genPartitionHdfsPath(ptns) - val dirPath = s"${filePath}/${ptnsPath}" - val dataFileName = s"${ms}" - val dataFilePath = HdfsUtil.getHdfsFilePath(dirPath, dataFileName) - - // encode data - val dataRdd: RDD[Row] = rdd.flatMap(encode(_, ms)) -// val dataRdd: RDD[CacheData] = rdd.flatMap(encode(_, ms)) - - // generate data frame - val df = sqlContext.createDataFrame(dataRdd, schema) -// val df = sqlContext.createDataFrame(dataRdd) - - // save data frame - df.write.parquet(dataFilePath) - } - // submit ms - submitCacheTime(ms) - submitReadyTime(ms) - } catch { - case e: Throwable => error(s"save data error: ${e.getMessage}") - } finally { - newCacheLock.unlock() - } - } - } - - def readData(): Try[RDD[Map[String, Any]]] = Try { - val timeRange = TimeInfoCache.getTimeRange - submitLastProcTime(timeRange._2) - - val reviseTimeRange = (timeRange._1 + deltaTimeRange._1, timeRange._2 + deltaTimeRange._2) - submitCleanTime(reviseTimeRange._1) - - // read directly through partition info - val partitionRanges = getPartitionRange(reviseTimeRange._1, reviseTimeRange._2) - val alterPartitionRanges = partitionRanges :+ (reviseTimeRange._1, reviseTimeRange._2) - println(alterPartitionRanges) - - // list partition paths - val partitionPaths = listPathsBetweenRanges(filePath :: Nil, alterPartitionRanges) - - if (partitionPaths.isEmpty) { - sqlContext.sparkContext.emptyRDD[Map[String, Any]] - } else { - val df = partitionPaths.map { path => - sqlContext.read.parquet(path) - }.reduce(_ unionAll _) - - // decode data - df.flatMap { row => - decode(row, true) - } - } - } - - override def cleanOldData(): Unit = { - val oldCacheLocked = oldCacheLock.lock(-1, TimeUnit.SECONDS) - if (oldCacheLocked) { - try { - val cleanTime = readCleanTime() - cleanTime match { - case Some(ct) => { - // drop partitions - val bounds = getPartition(ct) - val alterBounds = bounds :+ ct - - // list partition paths - val earlierPaths = listPathsEarlierThanBounds(filePath :: Nil, alterBounds) - - // delete out time data path - earlierPaths.foreach { path => - println(s"delete hdfs path: ${path}") - HdfsUtil.deleteHdfsPath(path) - } - } - case _ => { - // do nothing - } - } - } catch { - case e: Throwable => error(s"clean old data error: ${e.getMessage}") - } finally { - oldCacheLock.unlock() - } - } - } - -// override def updateOldData(t: Long, oldData: Iterable[Map[String, Any]]): Unit = { -// // parallel process different time groups, lock is unnecessary -// val ptns = getPartition(t) -// val ptnsPath = genPartitionHdfsPath(ptns) -// val dirPath = s"${filePath}/${ptnsPath}" -// val dataFileName = s"${t}" -// val dataFilePath = HdfsUtil.getHdfsFilePath(dirPath, dataFileName) -// -// try { -// // remove old data path -// HdfsUtil.deleteHdfsPath(dataFilePath) -// -// // save updated old data -// if (oldData.size > 0) { -// // encode data -// val recordDatas = oldData.flatMap { dt => -// encode(dt, t) -// }.toList -// -//// val rdd = sqlContext.sparkContext.parallelize(recordDatas) -// -// // generate data frame -// val df = sqlContext.createDataFrame(recordDatas, schema) -// -// // save data frame -// df.write.parquet(dataFilePath) -// } -// } catch { -// case e: Throwable => { -// error(s"update old data error: ${e.getMessage}") -// e.printStackTrace() -// } -// } -// } - - override protected def genCleanTime(ms: Long): Long = { - val minPartitionUnit = partitionUnits.last - val t1 = TimeUtil.timeToUnit(ms, minPartitionUnit) - val t2 = TimeUtil.timeFromUnit(t1, minPartitionUnit) - t2 - } - - private def getPartition(ms: Long): List[Long] = { - partitionUnits.map { unit => - TimeUtil.timeToUnit(ms, unit) - } - } - private def getPartitionRange(ms1: Long, ms2: Long): List[(Long, Long)] = { - partitionUnits.map { unit => - val t1 = TimeUtil.timeToUnit(ms1, unit) - val t2 = TimeUtil.timeToUnit(ms2, unit) - (t1, t2) - } - } - - private def genPartitionHdfsPath(partition: List[Long]): String = { - partition.map(prtn => s"${prtn}").mkString("/") - } - - private def str2Long(str: String): Option[Long] = { - try { - Some(str.toLong) - } catch { - case e: Throwable => None - } - } - - private def listPathsBetweenRanges(paths: List[String], - partitionRanges: List[(Long, Long)] - ): List[String] = { - partitionRanges match { - case Nil => paths - case head :: tail => { - val (lb, ub) = head - val curPaths = paths.flatMap { path => - val names = HdfsUtil.listSubPaths(path, "dir").toList - names.filter { name => - str2Long(name) match { - case Some(t) => (t >= lb) && (t <= ub) - case _ => false - } - }.map(HdfsUtil.getHdfsFilePath(path, _)) - } - listPathsBetweenRanges(curPaths, tail) - } - } - } - - private def listPathsEarlierThanBounds(paths: List[String], bounds: List[Long] - ): List[String] = { - bounds match { - case Nil => paths - case head :: tail => { - val earlierPaths = paths.flatMap { path => - val names = HdfsUtil.listSubPaths(path, "dir").toList - names.filter { name => - str2Long(name) match { - case Some(t) => (t < head) - case _ => false - } - }.map(HdfsUtil.getHdfsFilePath(path, _)) - } - val equalPaths = paths.flatMap { path => - val names = HdfsUtil.listSubPaths(path, "dir").toList - names.filter { name => - str2Long(name) match { - case Some(t) => (t == head) - case _ => false - } - }.map(HdfsUtil.getHdfsFilePath(path, _)) - } - earlierPaths ::: listPathsEarlierThanBounds(equalPaths, tail) - } - } - } -} From 68b1d53f11fd204403e6185e0d3af985d9e261a3 Mon Sep 17 00:00:00 2001 From: Lionel Liu Date: Thu, 10 Aug 2017 14:32:10 +0800 Subject: [PATCH 041/111] measure streaming sample doc --- griffin-doc/measure-streaming-sample.md | 204 ++++++++++++++++++++++++ 1 file changed, 204 insertions(+) create mode 100644 griffin-doc/measure-streaming-sample.md diff --git a/griffin-doc/measure-streaming-sample.md b/griffin-doc/measure-streaming-sample.md new file mode 100644 index 000000000..58ea74f11 --- /dev/null +++ b/griffin-doc/measure-streaming-sample.md @@ -0,0 +1,204 @@ + +# Measure streaming sample +Measures consists of batch measure and streaming measure. This document is for the streaming measure sample. + +### Data source +At current, we support kafka as streaming data source. +In this sample, we also need a kafka as data source. + +### Measure type +At current, we support accuracy measure in streaming mode. + +### Kafka decoder +In kafka, data always needs encode and decode, we support String type kafka data currently, you can also implement and use your decoder for kafka case. + +### Environment +For current griffin streaming case, we need some necessary environment dependencies, zookeeper and hdfs. +We use zookeeper to cache some checkpoint information, it's optional, but we recommend it. +We use hdfs to save the temporary data, it's also a recommend selection. + +### Streaming accuracy result +The streaming data will be separated into mini-batches of data, for each mini-batch data, there should be an accuracy result. Therefore, the streaming accuracy result should be a bunch of batch accuracy results with timestamp. +Considering the latency of streaming data, which means the source data and the matching target data will not exactly reach exactly at the same time, we have to accept some delay of data in streaming mode, by holding unmatched data in memory or disk, and try to match them later until the data is out-time. + +## How to run streaming sample +### Environment Preparation +At first, we need some environment preparation. +- Zookeeper: Zookeeper 3.4.10 +- Hadoop: Hadoop 2.6 +- Spark: Spark 1.6 +- Kafka: Kafka 0.8 + +### Data Preparation +Create two topics in kafka, for source and target data. For example, topic "source" for source data, and topic "target" for target data. +Streaming data should also be prepared, the format could be json string, for example: +Source data could be: +``` +{"name": "kevin", "age": 24} +{"name": "jason", "age": 25} +{"name": "jhon", "age": 28} +{"name": "steve", "age": 31} +``` +Target data could be: +``` +{"name": "kevin", "age": 24} +{"name": "jason", "age": 25} +{"name": "steve", "age": 20} +``` +You need to input the source data and target data into these two topics, through console producer might be a good choice for experimental purpose. + +### Configuration Preparation +Two configuration files are required. +Environment configuration file: env.json +``` +{ + "spark": { + "log.level": "WARN", + "checkpoint.dir": "hdfs:///griffin/streaming/cp", + "batch.interval": "5s", + "process.interval": "30s", + "config": { + "spark.task.maxFailures": 5, + "spark.streaming.kafkaMaxRatePerPartition": 1000, + "spark.streaming.concurrentJobs": 4 + } + }, + + "persist": [ + { + "type": "log", + "config": { + "max.log.lines": 100 + } + }, { + "type": "hdfs", + "config": { + "path": "hdfs:///griffin/streaming/persist", + "max.persist.lines": 10000, + "max.lines.per.file": 10000 + } + } + ], + + "info.cache": [ + { + "type": "zk", + "config": { + "hosts": ":2181", + "namespace": "griffin/infocache", + "lock.path": "lock", + "mode": "persist", + "init.clear": true, + "close.clear": false + } + } + ] +} +``` +In env.json, "spark" field configures the spark and spark streaming parameters, "persist" field configures the persist ways, we support "log", "hdfs" and "http" ways at current, "info.cache" field configures the information cache parameters, we support zookeeper only at current. + +Process configuration file: config.json +``` +{ + "name": "streaming-accu-sample", + "type": "accuracy", + "process.type": "streaming", + + "source": { + "type": "kafka", + "version": "0.8", + "config": { + "kafka.config": { + "bootstrap.servers": ":9092", + "group.id": "group1", + "auto.offset.reset": "smallest", + "auto.commit.enable": "false" + }, + "topics": "source", + "key.type": "java.lang.String", + "value.type": "java.lang.String" + }, + "cache": { + "type": "text", + "config": { + "file.path": "hdfs:///griffin/streaming/dump/source", + "info.path": "source", + "ready.time.interval": "10s", + "ready.time.delay": "0" + }, + "time.range": ["-5m", "0"] + }, + "match.once": true + }, + + "target": { + "type": "kafka", + "version": "0.8", + "config": { + "kafka.config": { + "bootstrap.servers": ":9092", + "group.id": "group1", + "auto.offset.reset": "smallest", + "auto.commit.enable": "false" + }, + "topics": "target", + "key.type": "java.lang.String", + "value.type": "java.lang.String" + }, + "cache": { + "type": "text", + "config": { + "file.path": "hdfs:///griffin/streaming/dump/target", + "info.path": "target", + "ready.time.interval": "10s", + "ready.time.delay": "0" + }, + "time.range": ["-5m", "0"] + }, + "match.once": false + }, + + "evaluateRule": { + "rules": "$source.json().name = $target.json().name AND $source.json().age = $target.json().age" + } +} +``` +In config.json, "source" and "target" fields configure the data source parameters. +The "cache" field in data source configuration represents the temporary data cache way, at current we support "text" and "hive" ways. We recommend "text" way, it only depends on hdfs. "time.range" means that the data older than the lower bound should be considered as out-time, and the out-time data will not be calculated any more. +"match.once" represents the data from this data source could be matched only once or more times. +"evaluateRule.rule" configures the match rule between each source and target data. + +### Run +Build the measure package. +``` +mvn install +``` +Get the measure package ```measure--incubating-SNAPSHOT.jar```, rename it to ```griffin-measure.jar```. +Put measure package together with env.json and config.json. +Run the following command: +``` +spark-submit --class org.apache.griffin.measure.Application \ +--master yarn-client --queue default \ +griffin-measure.jar \ +env.json config.json local,local +``` +The first two parameters are the paths of env.json and config.json, the third parameter represents the file system type of the two configuration files, "local" or "hdfs" are both supported. + +The spark streaming application will be long-time running, you can get the results of each mini-batch of data, during the run-time, you can also input more data into source and target topics, to check the results of the later mini-batches. From f2ded2455c7ed10e7fe7ae023bc753d3408a5817 Mon Sep 17 00:00:00 2001 From: Lionel Liu Date: Thu, 10 Aug 2017 14:45:27 +0800 Subject: [PATCH 042/111] modify --- griffin-doc/measure-streaming-sample.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/griffin-doc/measure-streaming-sample.md b/griffin-doc/measure-streaming-sample.md index 58ea74f11..004ed3ba4 100644 --- a/griffin-doc/measure-streaming-sample.md +++ b/griffin-doc/measure-streaming-sample.md @@ -188,7 +188,7 @@ The "cache" field in data source configuration represents the temporary data cac ### Run Build the measure package. ``` -mvn install +mvn clean install ``` Get the measure package ```measure--incubating-SNAPSHOT.jar```, rename it to ```griffin-measure.jar```. Put measure package together with env.json and config.json. From 527fa1365718147c4a34636cfc0cbd9b3c66b028 Mon Sep 17 00:00:00 2001 From: Lionel Liu Date: Thu, 10 Aug 2017 17:49:33 +0800 Subject: [PATCH 043/111] tst --- .../cache/DfCacheDataConnector.scala | 281 +++++++++++++ .../cache/ParquetCacheDataConnector.scala | 370 ++++++++++++++++++ 2 files changed, 651 insertions(+) create mode 100644 measure/src/main/scala/org/apache/griffin/measure/connector/cache/DfCacheDataConnector.scala create mode 100644 measure/src/main/scala/org/apache/griffin/measure/connector/cache/ParquetCacheDataConnector.scala diff --git a/measure/src/main/scala/org/apache/griffin/measure/connector/cache/DfCacheDataConnector.scala b/measure/src/main/scala/org/apache/griffin/measure/connector/cache/DfCacheDataConnector.scala new file mode 100644 index 000000000..f883b4fec --- /dev/null +++ b/measure/src/main/scala/org/apache/griffin/measure/connector/cache/DfCacheDataConnector.scala @@ -0,0 +1,281 @@ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ +package org.apache.griffin.measure.connector.cache + +import java.util.concurrent.TimeUnit + +import org.apache.griffin.measure.cache.info.{InfoCacheInstance, TimeInfoCache} +import org.apache.griffin.measure.config.params.user.DataCacheParam +import org.apache.griffin.measure.result.TimeStampInfo +import org.apache.griffin.measure.rule.DataTypeCalculationUtil +import org.apache.griffin.measure.utils.TimeUtil +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.types._ +import org.apache.spark.sql.{DataFrame, Row, SQLContext} +import org.apache.spark.storage.StorageLevel + +import scala.util.Try + +case class DfCacheDataConnector(sqlContext: SQLContext, dataCacheParam: DataCacheParam + ) extends CacheDataConnector { + + val config = dataCacheParam.config + val InfoPath = "info.path" + val cacheInfoPath: String = config.getOrElse(InfoPath, defCacheInfoPath).toString + + val newCacheLock = InfoCacheInstance.genLock(s"${cacheInfoPath}.new") + val oldCacheLock = InfoCacheInstance.genLock(s"${cacheInfoPath}.old") + + val timeRangeParam: List[String] = if (dataCacheParam.timeRange != null) dataCacheParam.timeRange else Nil + val deltaTimeRange: (Long, Long) = (timeRangeParam ::: List("0", "0")) match { + case s :: e :: _ => { + val ns = TimeUtil.milliseconds(s) match { + case Some(n) if (n < 0) => n + case _ => 0 + } + val ne = TimeUtil.milliseconds(e) match { + case Some(n) if (n < 0) => n + case _ => 0 + } + (ns, ne) + } + case _ => (0, 0) + } + + val CacheLevel = "cache.level" + val cacheLevel: String = config.getOrElse(CacheLevel, "MEMORY_AND_DISK").toString + + val timeStampColumn = TimeStampInfo.key + + var newDataFrame: DataFrame = null + var oldDataFrame: DataFrame = null + + val ReadyTimeInterval = "ready.time.interval" + val ReadyTimeDelay = "ready.time.delay" + val readyTimeInterval: Long = TimeUtil.milliseconds(config.getOrElse(ReadyTimeInterval, "1m").toString).getOrElse(60000L) + val readyTimeDelay: Long = TimeUtil.milliseconds(config.getOrElse(ReadyTimeDelay, "1m").toString).getOrElse(60000L) + + def available(): Boolean = { + true + } + + def init(): Unit = {} + + def saveData(rdd: RDD[Map[String, Any]], ms: Long): Unit = { + val newCacheLocked = newCacheLock.lock(-1, TimeUnit.SECONDS) + if (newCacheLocked) { + try { + if (newDataFrame == null) { + if (!rdd.isEmpty) { + newDataFrame = genDataFrame(rdd) + newDataFrame.persist(StorageLevel.fromString(cacheLevel)) + } + } else { + if (!rdd.isEmpty) { + newDataFrame.unpersist() + newDataFrame = newDataFrame.unionAll(genDataFrame(rdd)) + newDataFrame.persist(StorageLevel.fromString(cacheLevel)) + } + } + // submit ms + submitCacheTime(ms) + submitReadyTime(ms) + } catch { + case e: Throwable => error(s"save data error: ${e.getMessage}") + } finally { + newCacheLock.unlock() + } + } + } + + def readData(): Try[RDD[Map[String, Any]]] = Try { + val timeRange = TimeInfoCache.getTimeRange + submitLastProcTime(timeRange._2) + val reviseTimeRange = (timeRange._1 + deltaTimeRange._1, timeRange._2 + deltaTimeRange._2) + + // move new data frame to temp data frame + val newCacheLocked = newCacheLock.lock(-1, TimeUnit.SECONDS) + val newTempDataFrame = if (newCacheLocked) { + try { + val tmp = newDataFrame.filter(s"${timeStampColumn} BETWEEN ${reviseTimeRange._1} AND ${reviseTimeRange._2}") + newDataFrame.unpersist() + newDataFrame = newDataFrame.filter(s"${timeStampColumn} > ${reviseTimeRange._2}") + tmp + } catch { + case _ => null + } finally { + newCacheLock.unlock() + } + } else null + + // add temp data frame to old data frame + val oldCacheLocked = oldCacheLock.lock(-1, TimeUnit.SECONDS) + val oldTempDataFrame = if (oldCacheLocked) { + try { + if (oldDataFrame != null) { + oldDataFrame.filter(s"${timeStampColumn} BETWEEN ${reviseTimeRange._1} AND ${reviseTimeRange._2}") + } else null + } catch { + case _ => null + } finally { + oldCacheLock.unlock() + } + } else null + + val resultDataFrame = if (oldTempDataFrame == null && newTempDataFrame == null) { + throw new Exception("data not cached") + } else { + val finalDataFrame = if (newTempDataFrame == null) { + oldTempDataFrame + } else if (oldTempDataFrame == null) { + newTempDataFrame + } else { + oldTempDataFrame.unionAll(newTempDataFrame) + } + finalDataFrame + } + + // data frame -> rdd + resultDataFrame.map { row => + SparkRowFormatter.formatRow(row) + } + } + + override def cleanOldData(): Unit = { + val oldCacheLocked = oldCacheLock.lock(-1, TimeUnit.SECONDS) + if (oldCacheLocked) { + try { + val timeRange = TimeInfoCache.getTimeRange + val reviseTimeRange = (timeRange._1 + deltaTimeRange._1, timeRange._2 + deltaTimeRange._2) + + oldDataFrame.unpersist() + oldDataFrame = oldDataFrame.filter(s"${timeStampColumn} >= ${reviseTimeRange._1}") + oldDataFrame.persist(StorageLevel.fromString(cacheLevel)) + } catch { + case e: Throwable => error(s"clean old data error: ${e.getMessage}") + } finally { + oldCacheLock.unlock() + } + } + +// if (initialed) { +// val timeRange = TimeInfoCache.getTimeRange +// val reviseTimeRange = (timeRange._1 + deltaTimeRange._1, timeRange._2 + deltaTimeRange._2) +// println(s"clean reviseTimeRange: ${reviseTimeRange}") +// +// dataFrame.show(10) +// +// dataFrame.unpersist() +// dataFrame = dataFrame.filter(s"${timeStampColumn} >= ${reviseTimeRange._1}") +// dataFrame.persist(StorageLevel.fromString(cacheLevel)) +// +// dataFrame.show(10) +// } + } + + override def updateAllOldData(oldRdd: RDD[Map[String, Any]]): Unit = { + val oldCacheLocked = oldCacheLock.lock(-1, TimeUnit.SECONDS) + if (oldCacheLocked) { + try { + if (oldDataFrame == null) { + if (!oldRdd.isEmpty) { + oldDataFrame = genDataFrame(oldRdd) + oldDataFrame.persist(StorageLevel.fromString(cacheLevel)) + } + } else { + if (!oldRdd.isEmpty) { + oldDataFrame.unpersist() + oldDataFrame = genDataFrame(oldRdd) + oldDataFrame.persist(StorageLevel.fromString(cacheLevel)) + } else { + oldDataFrame.unpersist() + oldDataFrame = null + } + } + } catch { + case e: Throwable => error(s"update all old data error: ${e.getMessage}") + } finally { + oldCacheLock.unlock() + } + } + } + + // generate DataFrame + // maybe we can directly use def createDataFrame[A <: Product : TypeTag](rdd: RDD[A]): DataFrame + // to avoid generate data type by myself, just translate each value into Product + private def genDataFrame(rdd: RDD[Map[String, Any]]): DataFrame = { + val fields = rdd.aggregate(Map[String, DataType]())( + DataTypeCalculationUtil.sequenceDataTypeMap, DataTypeCalculationUtil.combineDataTypeMap + ).toList.map(f => StructField(f._1, f._2)) + val schema = StructType(fields) + val datas: RDD[Row] = rdd.map { d => + val values = fields.map { field => + val StructField(k, dt, _, _) = field + d.get(k) match { + case Some(v) => v + case _ => null + } + } + Row(values: _*) + } + val df = sqlContext.createDataFrame(datas, schema) + df + } + +} + +import scala.collection.mutable.ArrayBuffer + +object SparkRowFormatter { + + def formatRow(row: Row): Map[String, Any] = { + formatRowWithSchema(row, row.schema) + } + + private def formatRowWithSchema(row: Row, schema: StructType): Map[String, Any] = { + formatStruct(schema.fields, row) + } + + private def formatStruct(schema: Seq[StructField], r: Row) = { + val paired = schema.zip(r.toSeq) + paired.foldLeft(Map[String, Any]())((s, p) => s ++ formatItem(p)) + } + + private def formatItem(p: Pair[StructField, Any]): Map[String, Any] = { + p match { + case (sf, a) => + sf.dataType match { + case ArrayType(et, _) => + Map(sf.name -> (if (a == null) a else formatArray(et, a.asInstanceOf[ArrayBuffer[Any]]))) + case StructType(s) => + Map(sf.name -> (if (a == null) a else formatStruct(s, a.asInstanceOf[Row]))) + case _ => Map(sf.name -> a) + } + } + } + + private def formatArray(et: DataType, arr: ArrayBuffer[Any]): Seq[Any] = { + et match { + case StructType(s) => arr.map(e => formatStruct(s, e.asInstanceOf[Row])) + case ArrayType(t, _) => + arr.map(e => formatArray(t, e.asInstanceOf[ArrayBuffer[Any]])) + case _ => arr + } + } +} \ No newline at end of file diff --git a/measure/src/main/scala/org/apache/griffin/measure/connector/cache/ParquetCacheDataConnector.scala b/measure/src/main/scala/org/apache/griffin/measure/connector/cache/ParquetCacheDataConnector.scala new file mode 100644 index 000000000..b641d22fe --- /dev/null +++ b/measure/src/main/scala/org/apache/griffin/measure/connector/cache/ParquetCacheDataConnector.scala @@ -0,0 +1,370 @@ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ +package org.apache.griffin.measure.connector.cache + +import java.util.concurrent.TimeUnit + +import org.apache.griffin.measure.cache.info.{InfoCacheInstance, TimeInfoCache} +import org.apache.griffin.measure.config.params.user.DataCacheParam +import org.apache.griffin.measure.result.TimeStampInfo +import org.apache.griffin.measure.utils.{HdfsUtil, JsonUtil, TimeUtil} +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.{Row, SQLContext} +import org.apache.spark.sql.types._ +import scala.collection.JavaConversions._ + +import scala.util.Try + +case class ParquetCacheDataConnector(sqlContext: SQLContext, dataCacheParam: DataCacheParam + ) extends CacheDataConnector { + + val config = dataCacheParam.config + val InfoPath = "info.path" + val cacheInfoPath: String = config.getOrElse(InfoPath, defCacheInfoPath).toString + + val newCacheLock = InfoCacheInstance.genLock(s"${cacheInfoPath}.new") + val oldCacheLock = InfoCacheInstance.genLock(s"${cacheInfoPath}.old") + + val timeRangeParam: List[String] = if (dataCacheParam.timeRange != null) dataCacheParam.timeRange else Nil + val deltaTimeRange: (Long, Long) = (timeRangeParam ::: List("0", "0")) match { + case s :: e :: _ => { + val ns = TimeUtil.milliseconds(s) match { + case Some(n) if (n < 0) => n + case _ => 0 + } + val ne = TimeUtil.milliseconds(e) match { + case Some(n) if (n < 0) => n + case _ => 0 + } + (ns, ne) + } + case _ => (0, 0) + } + + val FilePath = "file.path" + val filePath: String = config.get(FilePath) match { + case Some(s: String) => s + case _ => throw new Exception("invalid file.path!") + } + + val ReadyTimeInterval = "ready.time.interval" + val ReadyTimeDelay = "ready.time.delay" + val readyTimeInterval: Long = TimeUtil.milliseconds(config.getOrElse(ReadyTimeInterval, "1m").toString).getOrElse(60000L) + val readyTimeDelay: Long = TimeUtil.milliseconds(config.getOrElse(ReadyTimeDelay, "1m").toString).getOrElse(60000L) + + val TimeStampColumn: String = TimeStampInfo.key + val PayloadColumn: String = "payload" + + // cache schema: Long, String + val fields = List[StructField]( + StructField(TimeStampColumn, LongType), + StructField(PayloadColumn, StringType) + ) + val schema = StructType(fields) + +// case class CacheData(time: Long, payload: String) { +// def getTime(): Long = time +// def getPayload(): String = payload +// } + + val partitionUnits: List[String] = List("hour", "min") + + override def init(): Unit = { + // do nothing + } + + def available(): Boolean = { + true + } + + private def encode(data: Map[String, Any], ms: Long): Option[Row] = { + try { + val values = fields.map { field => + val StructField(name, _, _, _) = field + name match { + case TimeStampColumn => ms + case PayloadColumn => JsonUtil.toJson(data) + case _ => null + } + } + Some(Row(values: _*)) + } catch { + case _ => None + } + } + + private def decode(data: Row, updateTimeStamp: Boolean): Option[Map[String, Any]] = { + data.getAs[Any](PayloadColumn) match { + case v: String => { + try { + val map = JsonUtil.toAnyMap(v) + val resMap = if (updateTimeStamp) { + data.getAs[Any](TimeStampColumn) match { + case t: Long => map + (TimeStampColumn -> t) + case _ => map + } + } else map + Some(resMap) + } catch { + case _ => None + } + } + case _ => None + } + } + +// private def encode(data: Map[String, Any], ms: Long): Option[CacheData] = { +// try { +// val json = JsonUtil.toJson(data) +// Some(CacheData(ms, json)) +// } catch { +// case _ => None +// } +// } +// +// private def decode(data: Row, updateTimeStamp: Boolean): Option[Map[String, Any]] = { +// data.getAs[Any]("payload") match { +// case v: String => { +// try { +// val map = JsonUtil.toAnyMap(v) +// val resMap = if (updateTimeStamp) { +// data.getAs[Any]("time") match { +// case t: Long => map + (TimeStampInfo.key -> t) +// case _ => map +// } +// } else map +// Some(resMap) +// } catch { +// case _ => None +// } +// } +// case _ => None +// } +// } + + def saveData(rdd: RDD[Map[String, Any]], ms: Long): Unit = { + val newCacheLocked = newCacheLock.lock(-1, TimeUnit.SECONDS) + if (newCacheLocked) { + try { + if (!rdd.isEmpty) { + val ptns = getPartition(ms) + val ptnsPath = genPartitionHdfsPath(ptns) + val dirPath = s"${filePath}/${ptnsPath}" + val dataFileName = s"${ms}" + val dataFilePath = HdfsUtil.getHdfsFilePath(dirPath, dataFileName) + + // encode data + val dataRdd: RDD[Row] = rdd.flatMap(encode(_, ms)) +// val dataRdd: RDD[CacheData] = rdd.flatMap(encode(_, ms)) + + // generate data frame + val df = sqlContext.createDataFrame(dataRdd, schema) +// val df = sqlContext.createDataFrame(dataRdd) + + // save data frame + df.write.parquet(dataFilePath) + } + // submit ms + submitCacheTime(ms) + submitReadyTime(ms) + } catch { + case e: Throwable => error(s"save data error: ${e.getMessage}") + } finally { + newCacheLock.unlock() + } + } + } + + def readData(): Try[RDD[Map[String, Any]]] = Try { + val timeRange = TimeInfoCache.getTimeRange + submitLastProcTime(timeRange._2) + + val reviseTimeRange = (timeRange._1 + deltaTimeRange._1, timeRange._2 + deltaTimeRange._2) + submitCleanTime(reviseTimeRange._1) + + // read directly through partition info + val partitionRanges = getPartitionRange(reviseTimeRange._1, reviseTimeRange._2) + val alterPartitionRanges = partitionRanges :+ (reviseTimeRange._1, reviseTimeRange._2) + println(alterPartitionRanges) + + // list partition paths + val partitionPaths = listPathsBetweenRanges(filePath :: Nil, alterPartitionRanges) + + if (partitionPaths.isEmpty) { + sqlContext.sparkContext.emptyRDD[Map[String, Any]] + } else { + val df = partitionPaths.map { path => + sqlContext.read.parquet(path) + }.reduce(_ unionAll _) + + // decode data + df.flatMap { row => + decode(row, true) + } + } + } + + override def cleanOldData(): Unit = { + val oldCacheLocked = oldCacheLock.lock(-1, TimeUnit.SECONDS) + if (oldCacheLocked) { + try { + val cleanTime = readCleanTime() + cleanTime match { + case Some(ct) => { + // drop partitions + val bounds = getPartition(ct) + val alterBounds = bounds :+ ct + + // list partition paths + val earlierPaths = listPathsEarlierThanBounds(filePath :: Nil, alterBounds) + + // delete out time data path + earlierPaths.foreach { path => + println(s"delete hdfs path: ${path}") + HdfsUtil.deleteHdfsPath(path) + } + } + case _ => { + // do nothing + } + } + } catch { + case e: Throwable => error(s"clean old data error: ${e.getMessage}") + } finally { + oldCacheLock.unlock() + } + } + } + +// override def updateOldData(t: Long, oldData: Iterable[Map[String, Any]]): Unit = { +// // parallel process different time groups, lock is unnecessary +// val ptns = getPartition(t) +// val ptnsPath = genPartitionHdfsPath(ptns) +// val dirPath = s"${filePath}/${ptnsPath}" +// val dataFileName = s"${t}" +// val dataFilePath = HdfsUtil.getHdfsFilePath(dirPath, dataFileName) +// +// try { +// // remove old data path +// HdfsUtil.deleteHdfsPath(dataFilePath) +// +// // save updated old data +// if (oldData.size > 0) { +// // encode data +// val recordDatas = oldData.flatMap { dt => +// encode(dt, t) +// }.toList +// +//// val rdd = sqlContext.sparkContext.parallelize(recordDatas) +// +// // generate data frame +// val df = sqlContext.createDataFrame(recordDatas, schema) +// +// // save data frame +// df.write.parquet(dataFilePath) +// } +// } catch { +// case e: Throwable => { +// error(s"update old data error: ${e.getMessage}") +// e.printStackTrace() +// } +// } +// } + + override protected def genCleanTime(ms: Long): Long = { + val minPartitionUnit = partitionUnits.last + val t1 = TimeUtil.timeToUnit(ms, minPartitionUnit) + val t2 = TimeUtil.timeFromUnit(t1, minPartitionUnit) + t2 + } + + private def getPartition(ms: Long): List[Long] = { + partitionUnits.map { unit => + TimeUtil.timeToUnit(ms, unit) + } + } + private def getPartitionRange(ms1: Long, ms2: Long): List[(Long, Long)] = { + partitionUnits.map { unit => + val t1 = TimeUtil.timeToUnit(ms1, unit) + val t2 = TimeUtil.timeToUnit(ms2, unit) + (t1, t2) + } + } + + private def genPartitionHdfsPath(partition: List[Long]): String = { + partition.map(prtn => s"${prtn}").mkString("/") + } + + private def str2Long(str: String): Option[Long] = { + try { + Some(str.toLong) + } catch { + case e: Throwable => None + } + } + + private def listPathsBetweenRanges(paths: List[String], + partitionRanges: List[(Long, Long)] + ): List[String] = { + partitionRanges match { + case Nil => paths + case head :: tail => { + val (lb, ub) = head + val curPaths = paths.flatMap { path => + val names = HdfsUtil.listSubPaths(path, "dir").toList + names.filter { name => + str2Long(name) match { + case Some(t) => (t >= lb) && (t <= ub) + case _ => false + } + }.map(HdfsUtil.getHdfsFilePath(path, _)) + } + listPathsBetweenRanges(curPaths, tail) + } + } + } + + private def listPathsEarlierThanBounds(paths: List[String], bounds: List[Long] + ): List[String] = { + bounds match { + case Nil => paths + case head :: tail => { + val earlierPaths = paths.flatMap { path => + val names = HdfsUtil.listSubPaths(path, "dir").toList + names.filter { name => + str2Long(name) match { + case Some(t) => (t < head) + case _ => false + } + }.map(HdfsUtil.getHdfsFilePath(path, _)) + } + val equalPaths = paths.flatMap { path => + val names = HdfsUtil.listSubPaths(path, "dir").toList + names.filter { name => + str2Long(name) match { + case Some(t) => (t == head) + case _ => false + } + }.map(HdfsUtil.getHdfsFilePath(path, _)) + } + earlierPaths ::: listPathsEarlierThanBounds(equalPaths, tail) + } + } + } +} From 2db956feedd29caf21811ef3d2eff5a2b9434570 Mon Sep 17 00:00:00 2001 From: Lionel Liu Date: Fri, 11 Aug 2017 14:35:07 +0800 Subject: [PATCH 044/111] t1 --- measure/src/test/resources/test-data.jsonFile | 3 + measure/src/test/resources/test-data0.json | 56 ++++++++ .../src/test/resources/test-data1.jsonFile | 31 +++++ .../apache/griffin/measure/sql/SqlTest.scala | 125 ++++++++++++++++++ 4 files changed, 215 insertions(+) create mode 100644 measure/src/test/resources/test-data.jsonFile create mode 100644 measure/src/test/resources/test-data0.json create mode 100644 measure/src/test/resources/test-data1.jsonFile create mode 100644 measure/src/test/scala/org/apache/griffin/measure/sql/SqlTest.scala diff --git a/measure/src/test/resources/test-data.jsonFile b/measure/src/test/resources/test-data.jsonFile new file mode 100644 index 000000000..73707f45c --- /dev/null +++ b/measure/src/test/resources/test-data.jsonFile @@ -0,0 +1,3 @@ +{ "name": "emily", "age": 5, "map": { "a": 1, "b": 2 }, "list": [ { "c": 1, "d": 2 }, { "c": 3, "d": 4 } ], "t": [1, 2, 3] } +{ "name": "white", "age": 15, "map": { "a": 11, "b": 12 }, "list": [ { "c": 11, "d": 2 }, { "c": 23, "d": 4 } ], "t": [1, 2, 3] } +{ "name": "west", "age": 25, "map": { "a": 21, "b": 22 }, "list": [ { "c": 11, "d": 2 }, { "c": 23, "d": 4 } ], "t": [1, 2, 3] } \ No newline at end of file diff --git a/measure/src/test/resources/test-data0.json b/measure/src/test/resources/test-data0.json new file mode 100644 index 000000000..406acb87b --- /dev/null +++ b/measure/src/test/resources/test-data0.json @@ -0,0 +1,56 @@ +[ + { + "name": "emily", + "age": 5, + "map": { + "a": 1, + "b": 2 + }, + "list": [ + { + "c": 1, + "d": 2 + }, + { + "c": 3, + "d": 4 + } + ] + }, + { + "name": "white", + "age": 15, + "map": { + "a": 11, + "b": 12 + }, + "list": [ + { + "c": 11, + "d": 2 + }, + { + "c": 23, + "d": 4 + } + ] + }, + { + "name": "west", + "age": 25, + "map": { + "a": 21, + "b": 22 + }, + "list": [ + { + "c": 11, + "d": 2 + }, + { + "c": 23, + "d": 4 + } + ] + } +] \ No newline at end of file diff --git a/measure/src/test/resources/test-data1.jsonFile b/measure/src/test/resources/test-data1.jsonFile new file mode 100644 index 000000000..1e1f28a88 --- /dev/null +++ b/measure/src/test/resources/test-data1.jsonFile @@ -0,0 +1,31 @@ +[{ + "Year": "2013", + "First Name": "DAVID", + "County": "KINGS", + "Sex": "M", + "Count": "272" +}, { + "Year": "2013", + "First Name": "JAYDEN", + "County": "KINGS", + "Sex": "M", + "Count": "268" +}, { + "Year": "2013", + "First Name": "JAYDEN", + "County": "QUEENS", + "Sex": "M", + "Count": "219" +}, { + "Year": "2013", + "First Name": "MOSHE", + "County": "KINGS", + "Sex": "M", + "Count": "219" +}, { + "Year": "2013", + "First Name": "ETHAN", + "County": "QUEENS", + "Sex": "M", + "Count": "216" +}] \ No newline at end of file diff --git a/measure/src/test/scala/org/apache/griffin/measure/sql/SqlTest.scala b/measure/src/test/scala/org/apache/griffin/measure/sql/SqlTest.scala new file mode 100644 index 000000000..1e05198a9 --- /dev/null +++ b/measure/src/test/scala/org/apache/griffin/measure/sql/SqlTest.scala @@ -0,0 +1,125 @@ +package org.apache.griffin.measure.sql + +import org.apache.griffin.measure.config.params.user.EvaluateRuleParam +import org.apache.griffin.measure.rule.expr.{Expr, StatementExpr} +import org.apache.spark.sql.{DataFrame, SQLContext} +import org.apache.spark.sql.types.{ArrayType, IntegerType, StructField, StructType} +import org.apache.spark.{SparkConf, SparkContext} +import org.junit.runner.RunWith +import org.scalatest.junit.JUnitRunner +import org.scalatest.{BeforeAndAfter, FunSuite, Matchers} + +@RunWith(classOf[JUnitRunner]) +class SqlTest extends FunSuite with BeforeAndAfter with Matchers { + + var sc: SparkContext = _ + var sqlContext: SQLContext = _ + + before { + val conf = new SparkConf().setMaster("local[*]").setAppName("test") + sc = new SparkContext(conf) + sqlContext = new SQLContext(sc) + } + + test ("spark sql") { + + val squared = (s: Int) => { + s * s + } + sqlContext.udf.register("square", squared) + + val a = sqlContext.range(1, 20) + a.show + + a.registerTempTable("test") + + val table = sqlContext.sql("select * from test") + table.show() + + val result = sqlContext.sql("select id, square(id) as id_squared from test") + result.show() + + } + + test ("json") { + def jsonToDataFrame(json: String, schema: Option[StructType] = None): DataFrame = { + val reader = sqlContext.read + val rd = schema match { + case Some(scm) => reader.schema(scm) + case _ => reader + } + rd.json(sc.parallelize(json :: Nil)) + } + + val json = + """ + |{ + | "a": [ + | 1, 2, 3 + | ] + |} + """.stripMargin + +// val bt = StructField("b", IntegerType) +// val at = StructField("a", StructType(bt :: Nil)) +// val schema = StructType(at :: Nil) + + val at = StructField("a", ArrayType(IntegerType)) + val schema = StructType(at :: Nil) + + val df = jsonToDataFrame(json, Some(schema)) + + df.registerTempTable("json") + + val result = sqlContext.sql("select a[1] from json") + result.show + + } + + test ("json file") { + + // read json file directly +// val filePath = "src/test/resources/test-data.jsonFile" +// val reader = sqlContext.read +// val df = reader.json(filePath) +// df.show +// +// df.registerTempTable("ttt") +// val result = sqlContext.sql("select * from ttt where list[0].c = 11") +// result.show + + // whole json file +// val filePath = "src/test/resources/test-data0.json" +//// val filePath = "hdfs://localhost/test/file/t1.json" +// val jsonRDD = sc.wholeTextFiles(s"${filePath},${filePath}").map(x => x._2) +// val namesJson = sqlContext.read.json(jsonRDD) +// namesJson.printSchema +// namesJson.show + + // read text file then convert to json + val filePath = "src/test/resources/test-data.jsonFile" + val rdd = sc.textFile(filePath) + val reader = sqlContext.read + val df = reader.json(rdd) + df.show + df.printSchema + + df.registerTempTable("ttt") + val result = sqlContext.sql("select * from ttt where list[0].c = 11") + result.show + + // udf + val slice = (arr: Seq[Long], f: Int, e: Int) => arr.slice(f, e) +// val slice = (arr: Seq[Long]) => arr.slice(0, 1) + sqlContext.udf.register("slice", slice) + + val result1 = sqlContext.sql("select slice(t, 0, 2) from ttt") + result1.show + + } + + test ("accu sql") { +// val file1 = + } + +} From 6f9a8c6494c3ec4d7e5f01e259942274b25d2ab9 Mon Sep 17 00:00:00 2001 From: Lionel Liu Date: Mon, 14 Aug 2017 10:47:29 +0800 Subject: [PATCH 045/111] v1 --- .../streaming/StreamingAccuracyAlgo.scala | 21 ++++++++++++++----- 1 file changed, 16 insertions(+), 5 deletions(-) diff --git a/measure/src/main/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgo.scala b/measure/src/main/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgo.scala index bdac64e0e..41142755a 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgo.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgo.scala @@ -59,12 +59,23 @@ case class StreamingAccuracyAlgo(allParam: AllParam) extends AccuracyAlgo { val sqlContext = new HiveContext(sc) // val sqlContext = new SQLContext(sc) - val batchInterval = TimeUtil.milliseconds(sparkParam.batchInterval) match { - case Some(interval) => Milliseconds(interval) - case _ => throw new Exception("invalid batch interval") +// val batchInterval = TimeUtil.milliseconds(sparkParam.batchInterval) match { +// case Some(interval) => Milliseconds(interval) +// case _ => throw new Exception("invalid batch interval") +// } +// val ssc = new StreamingContext(sc, batchInterval) +// ssc.checkpoint(sparkParam.cpDir) + + def createStreamingContext(): StreamingContext = { + val batchInterval = TimeUtil.milliseconds(sparkParam.batchInterval) match { + case Some(interval) => Milliseconds(interval) + case _ => throw new Exception("invalid batch interval") + } + val ssc = new StreamingContext(sc, batchInterval) + ssc.checkpoint(sparkParam.cpDir) + ssc } - val ssc = new StreamingContext(sc, batchInterval) - ssc.checkpoint(sparkParam.cpDir) + val ssc = StreamingContext.getOrCreate(sparkParam.cpDir, createStreamingContext) // init info cache instance InfoCacheInstance.initInstance(envParam.infoCacheParams, metricName) From f7a833d9ed18cb440d661ea14662ed14f7f35201 Mon Sep 17 00:00:00 2001 From: Lionel Liu Date: Mon, 14 Aug 2017 10:48:53 +0800 Subject: [PATCH 046/111] v1 --- .../streaming/StreamingAccuracyAlgoTest.scala | 506 +++++++++--------- 1 file changed, 250 insertions(+), 256 deletions(-) diff --git a/measure/src/test/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgoTest.scala b/measure/src/test/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgoTest.scala index a22f91f59..9ee896a45 100644 --- a/measure/src/test/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgoTest.scala +++ b/measure/src/test/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgoTest.scala @@ -1,267 +1,261 @@ -///* -//Licensed to the Apache Software Foundation (ASF) under one -//or more contributor license agreements. See the NOTICE file -//distributed with this work for additional information -//regarding copyright ownership. The ASF licenses this file -//to you under the Apache License, Version 2.0 (the -//"License"); you may not use this file except in compliance -//with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -//Unless required by applicable law or agreed to in writing, -//software distributed under the License is distributed on an -//"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -//KIND, either express or implied. See the License for the -//specific language governing permissions and limitations -//under the License. -//*/ -//package org.apache.griffin.measure.algo.streaming -// -//import java.util.Date -//import java.util.concurrent.TimeUnit -// -//import org.apache.griffin.measure.algo.batch.BatchAccuracyAlgo -//import org.apache.griffin.measure.cache.info.{InfoCacheInstance, TimeInfoCache} -//import org.apache.griffin.measure.cache.result._ -//import org.apache.griffin.measure.config.params._ -//import org.apache.griffin.measure.config.params.env._ -//import org.apache.griffin.measure.config.params.user._ -//import org.apache.griffin.measure.config.reader._ -//import org.apache.griffin.measure.config.validator._ -//import org.apache.griffin.measure.connector.direct.DirectDataConnector -//import org.apache.griffin.measure.connector.{DataConnector, DataConnectorFactory} -//import org.apache.griffin.measure.log.Loggable -//import org.apache.griffin.measure.persist.{Persist, PersistFactory, PersistType} -//import org.apache.griffin.measure.result._ -//import org.apache.griffin.measure.rule.expr._ -//import org.apache.griffin.measure.rule.{ExprValueUtil, RuleAnalyzer, RuleFactory} -//import org.apache.griffin.measure.utils.{HdfsUtil, TimeUtil} -//import org.apache.spark.rdd.RDD -//import org.apache.spark.sql.SQLContext -//import org.apache.spark.sql.hive.HiveContext -//import org.apache.spark.streaming.{Milliseconds, StreamingContext} -//import org.apache.spark.{SparkConf, SparkContext} -//import org.junit.runner.RunWith -//import org.scalatest.junit.JUnitRunner -//import org.scalatest.{BeforeAndAfter, FunSuite, Matchers} -// -//import scala.util.{Failure, Success, Try} -// -// -//@RunWith(classOf[JUnitRunner]) -//class StreamingAccuracyAlgoTest extends FunSuite with Matchers with BeforeAndAfter with Loggable { -// -// val envFile = "src/test/resources/env-streaming.json" -// val confFile = "src/test/resources/config-streaming3.json" -// val envFsType = "local" -// val userFsType = "local" -// -// val args = Array(envFile, confFile) -// -// var sc: SparkContext = _ -// var sqlContext: SQLContext = _ -//// val ssc: StreamingContext = _ -// -// var allParam: AllParam = _ -// -// before { -// // read param files -// val envParam = readParamFile[EnvParam](envFile, envFsType) match { -// case Success(p) => p -// case Failure(ex) => { -// error(ex.getMessage) -// sys.exit(-2) -// } -// } -// val userParam = readParamFile[UserParam](confFile, userFsType) match { -// case Success(p) => p -// case Failure(ex) => { -// error(ex.getMessage) -// sys.exit(-2) -// } -// } -// allParam = AllParam(envParam, userParam) -// -// // validate param files -// validateParams(allParam) match { -// case Failure(ex) => { -// error(ex.getMessage) -// sys.exit(-3) -// } -// case _ => { -// info("params validation pass") -// } -// } -// -// val metricName = userParam.name -// val sparkParam = envParam.sparkParam -// val conf = new SparkConf().setMaster("local[*]").setAppName(metricName) -// conf.setAll(sparkParam.config) -// sc = new SparkContext(conf) -// sc.setLogLevel(envParam.sparkParam.logLevel) -// sqlContext = new SQLContext(sc) -//// sqlContext = new HiveContext(sc) -// -//// val a = sqlContext.sql("select * from s1 limit 10") -//// // val a = sqlContext.sql("show tables") -//// a.show(10) -//// -//// val b = HdfsUtil.existPath("/griffin/streaming") -//// println(b) -// } -// -// test("algorithm") { -// val envParam = allParam.envParam -// val userParam = allParam.userParam -// val metricName = userParam.name -// val sparkParam = envParam.sparkParam -// val cleanerParam = envParam.cleanerParam -// -//// val ssc = StreamingContext.getOrCreate(sparkParam.cpDir, -//// ( ) => { -//// try { -//// val batchInterval = TimeUtil.milliseconds(sparkParam.batchInterval) match { -//// case Some(interval) => Milliseconds(interval) -//// case _ => throw new Exception("invalid batch interval") -//// } -//// val ssc = new StreamingContext(sc, batchInterval) -//// ssc.checkpoint(sparkParam.cpDir) -//// ssc -//// } catch { -//// case runtime: RuntimeException => { -//// throw runtime -//// } -//// } -//// }) -// +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ +package org.apache.griffin.measure.algo.streaming + +import java.util.Date +import java.util.concurrent.TimeUnit + +import org.apache.griffin.measure.algo.batch.BatchAccuracyAlgo +import org.apache.griffin.measure.cache.info.{InfoCacheInstance, TimeInfoCache} +import org.apache.griffin.measure.cache.result._ +import org.apache.griffin.measure.config.params._ +import org.apache.griffin.measure.config.params.env._ +import org.apache.griffin.measure.config.params.user._ +import org.apache.griffin.measure.config.reader._ +import org.apache.griffin.measure.config.validator._ +import org.apache.griffin.measure.connector.direct.DirectDataConnector +import org.apache.griffin.measure.connector.{DataConnector, DataConnectorFactory} +import org.apache.griffin.measure.log.Loggable +import org.apache.griffin.measure.persist.{Persist, PersistFactory, PersistType} +import org.apache.griffin.measure.result._ +import org.apache.griffin.measure.rule.expr._ +import org.apache.griffin.measure.rule.{ExprValueUtil, RuleAnalyzer, RuleFactory} +import org.apache.griffin.measure.utils.{HdfsUtil, TimeUtil} +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.hive.HiveContext +import org.apache.spark.streaming.{Milliseconds, StreamingContext} +import org.apache.spark.{SparkConf, SparkContext} +import org.junit.runner.RunWith +import org.scalatest.junit.JUnitRunner +import org.scalatest.{BeforeAndAfter, FunSuite, Matchers} + +import scala.util.{Failure, Success, Try} + + +@RunWith(classOf[JUnitRunner]) +class StreamingAccuracyAlgoTest extends FunSuite with Matchers with BeforeAndAfter with Loggable { + + val envFile = "src/test/resources/env-streaming.json" + val confFile = "src/test/resources/config-streaming3.json" + val envFsType = "local" + val userFsType = "local" + + val args = Array(envFile, confFile) + + var sc: SparkContext = _ + var sqlContext: SQLContext = _ +// val ssc: StreamingContext = _ + + var allParam: AllParam = _ + + before { + // read param files + val envParam = readParamFile[EnvParam](envFile, envFsType) match { + case Success(p) => p + case Failure(ex) => { + error(ex.getMessage) + sys.exit(-2) + } + } + val userParam = readParamFile[UserParam](confFile, userFsType) match { + case Success(p) => p + case Failure(ex) => { + error(ex.getMessage) + sys.exit(-2) + } + } + allParam = AllParam(envParam, userParam) + + // validate param files + validateParams(allParam) match { + case Failure(ex) => { + error(ex.getMessage) + sys.exit(-3) + } + case _ => { + info("params validation pass") + } + } + + val metricName = userParam.name + val sparkParam = envParam.sparkParam + val conf = new SparkConf().setMaster("local[*]").setAppName(metricName) + conf.setAll(sparkParam.config) + sc = new SparkContext(conf) + sc.setLogLevel(envParam.sparkParam.logLevel) + sqlContext = new SQLContext(sc) +// sqlContext = new HiveContext(sc) + +// val a = sqlContext.sql("select * from s1 limit 10") +// // val a = sqlContext.sql("show tables") +// a.show(10) +// +// val b = HdfsUtil.existPath("/griffin/streaming") +// println(b) + } + + test("algorithm") { + val envParam = allParam.envParam + val userParam = allParam.userParam + val metricName = userParam.name + val sparkParam = envParam.sparkParam + val cleanerParam = envParam.cleanerParam + // val batchInterval = TimeUtil.milliseconds(sparkParam.batchInterval) match { // case Some(interval) => Milliseconds(interval) // case _ => throw new Exception("invalid batch interval") // } // val ssc = new StreamingContext(sc, batchInterval) // ssc.checkpoint(sparkParam.cpDir) -// -// // start time -// val startTime = new Date().getTime() -// -// val persistFactory = PersistFactory(envParam.persistParams, metricName) -// -// // get persists to persist measure result -// val appPersist: Persist = persistFactory.getPersists(startTime) -// -// // get spark application id -// val applicationId = sc.applicationId -// -// // persist start id -// appPersist.start(applicationId) -// -// InfoCacheInstance.initInstance(envParam.infoCacheParams, metricName) -// InfoCacheInstance.init -// -// // generate rule from rule param, generate rule analyzer -// val ruleFactory = RuleFactory(userParam.evaluateRuleParam) -// val rule: StatementExpr = ruleFactory.generateRule() -// val ruleAnalyzer: RuleAnalyzer = RuleAnalyzer(rule) -// -// // const expr value map -// val constExprValueMap = ExprValueUtil.genExprValueMaps(None, ruleAnalyzer.constCacheExprs, Map[String, Any]()) -// val finalConstExprValueMap = ExprValueUtil.updateExprValueMaps(ruleAnalyzer.constFinalCacheExprs, constExprValueMap) -// val finalConstMap = finalConstExprValueMap.headOption match { -// case Some(m) => m -// case _ => Map[String, Any]() -// } -// -// // data connector -// val sourceDataConnector: DirectDataConnector = -// DataConnectorFactory.getDirectDataConnector(sqlContext, ssc, userParam.sourceParam, -// ruleAnalyzer.sourceRuleExprs, finalConstMap -// ) match { -// case Success(cntr) => { -// if (cntr.available) cntr -// else throw new Exception("source data connection error!") -// } -// case Failure(ex) => throw ex -// } -// val targetDataConnector: DirectDataConnector = -// DataConnectorFactory.getDirectDataConnector(sqlContext, ssc, userParam.targetParam, -// ruleAnalyzer.targetRuleExprs, finalConstMap -// ) match { -// case Success(cntr) => { -// if (cntr.available) cntr -// else throw new Exception("target data connection error!") + + def createStreamingContext(): StreamingContext = { + val batchInterval = TimeUtil.milliseconds(sparkParam.batchInterval) match { + case Some(interval) => Milliseconds(interval) + case _ => throw new Exception("invalid batch interval") + } + val ssc = new StreamingContext(sc, batchInterval) + ssc.checkpoint(sparkParam.cpDir) + ssc + } + val ssc = StreamingContext.getOrCreate(sparkParam.cpDir, createStreamingContext) + + // start time + val startTime = new Date().getTime() + + val persistFactory = PersistFactory(envParam.persistParams, metricName) + + // get persists to persist measure result + val appPersist: Persist = persistFactory.getPersists(startTime) + + // get spark application id + val applicationId = sc.applicationId + + // persist start id + appPersist.start(applicationId) + + InfoCacheInstance.initInstance(envParam.infoCacheParams, metricName) + InfoCacheInstance.init + + // generate rule from rule param, generate rule analyzer + val ruleFactory = RuleFactory(userParam.evaluateRuleParam) + val rule: StatementExpr = ruleFactory.generateRule() + val ruleAnalyzer: RuleAnalyzer = RuleAnalyzer(rule) + + // const expr value map + val constExprValueMap = ExprValueUtil.genExprValueMaps(None, ruleAnalyzer.constCacheExprs, Map[String, Any]()) + val finalConstExprValueMap = ExprValueUtil.updateExprValueMaps(ruleAnalyzer.constFinalCacheExprs, constExprValueMap) + val finalConstMap = finalConstExprValueMap.headOption match { + case Some(m) => m + case _ => Map[String, Any]() + } + + // data connector + val sourceDataConnector: DirectDataConnector = + DataConnectorFactory.getDirectDataConnector(sqlContext, ssc, userParam.sourceParam, + ruleAnalyzer.sourceRuleExprs, finalConstMap + ) match { + case Success(cntr) => { + if (cntr.available) cntr + else throw new Exception("source data connection error!") + } + case Failure(ex) => throw ex + } + val targetDataConnector: DirectDataConnector = + DataConnectorFactory.getDirectDataConnector(sqlContext, ssc, userParam.targetParam, + ruleAnalyzer.targetRuleExprs, finalConstMap + ) match { + case Success(cntr) => { + if (cntr.available) cntr + else throw new Exception("target data connection error!") + } + case Failure(ex) => throw ex + } + + val cacheResultProcesser = CacheResultProcesser() + + // init data stream + sourceDataConnector.init() + targetDataConnector.init() + + // my algo + val algo = StreamingAccuracyAlgo(allParam) + + val streamingAccuracyProcess = StreamingAccuracyProcess( + sourceDataConnector, targetDataConnector, + ruleAnalyzer, cacheResultProcesser, persistFactory, appPersist) + + val processInterval = TimeUtil.milliseconds(sparkParam.processInterval) match { + case Some(interval) => interval + case _ => throw new Exception("invalid batch interval") + } + val process = TimingProcess(processInterval, streamingAccuracyProcess) + + // clean thread +// case class Clean() extends Runnable { +// val lock = InfoCacheInstance.genLock("clean") +// def run(): Unit = { +// val locked = lock.lock(5, TimeUnit.SECONDS) +// if (locked) { +// try { +// sourceDataConnector.cleanData +// targetDataConnector.cleanData +// } finally { +// lock.unlock() +// } // } -// case Failure(ex) => throw ex // } -// -// val cacheResultProcesser = CacheResultProcesser() -// -// // init data stream -// sourceDataConnector.init() -// targetDataConnector.init() -// -// // my algo -// val algo = StreamingAccuracyAlgo(allParam) -// -// val streamingAccuracyProcess = StreamingAccuracyProcess( -// sourceDataConnector, targetDataConnector, -// ruleAnalyzer, cacheResultProcesser, persistFactory, appPersist) -// -// val processInterval = TimeUtil.milliseconds(sparkParam.processInterval) match { +// } +// val cleanInterval = TimeUtil.milliseconds(cleanerParam.cleanInterval) match { // case Some(interval) => interval // case _ => throw new Exception("invalid batch interval") // } -// val process = TimingProcess(processInterval, streamingAccuracyProcess) -// -// // clean thread -//// case class Clean() extends Runnable { -//// val lock = InfoCacheInstance.genLock("clean") -//// def run(): Unit = { -//// val locked = lock.lock(5, TimeUnit.SECONDS) -//// if (locked) { -//// try { -//// sourceDataConnector.cleanData -//// targetDataConnector.cleanData -//// } finally { -//// lock.unlock() -//// } -//// } -//// } -//// } -//// val cleanInterval = TimeUtil.milliseconds(cleanerParam.cleanInterval) match { -//// case Some(interval) => interval -//// case _ => throw new Exception("invalid batch interval") -//// } -//// val clean = TimingProcess(cleanInterval, Clean()) -// -// process.startup() -//// clean.startup() -// -// ssc.start() -// ssc.awaitTermination() -// ssc.stop(stopSparkContext=true, stopGracefully=true) -// -// println("================ end ================") -// -// // context stop -// sc.stop -// -// InfoCacheInstance.close -// -// appPersist.finish() -// -// process.shutdown() -//// clean.shutdown() -// } -// -// private def readParamFile[T <: Param](file: String, fsType: String)(implicit m : Manifest[T]): Try[T] = { -// val paramReader = ParamReaderFactory.getParamReader(file, fsType) -// paramReader.readConfig[T] -// } -// -// private def validateParams(allParam: AllParam): Try[Boolean] = { -// val allParamValidator = AllParamValidator() -// allParamValidator.validate(allParam) -// } -// -//} +// val clean = TimingProcess(cleanInterval, Clean()) + + process.startup() +// clean.startup() + + ssc.start() + ssc.awaitTermination() + ssc.stop(stopSparkContext=true, stopGracefully=true) + + println("================ end ================") + + // context stop + sc.stop + + InfoCacheInstance.close + + appPersist.finish() + + process.shutdown() +// clean.shutdown() + } + + private def readParamFile[T <: Param](file: String, fsType: String)(implicit m : Manifest[T]): Try[T] = { + val paramReader = ParamReaderFactory.getParamReader(file, fsType) + paramReader.readConfig[T] + } + + private def validateParams(allParam: AllParam): Try[Boolean] = { + val allParamValidator = AllParamValidator() + allParamValidator.validate(allParam) + } + +} From f2320a57bd58026febdcdd858c7601826e7011d1 Mon Sep 17 00:00:00 2001 From: Lionel Liu Date: Mon, 14 Aug 2017 10:49:08 +0800 Subject: [PATCH 047/111] v1 --- .../streaming/StreamingAccuracyAlgoTest.scala | 510 +++++++++--------- 1 file changed, 255 insertions(+), 255 deletions(-) diff --git a/measure/src/test/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgoTest.scala b/measure/src/test/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgoTest.scala index 9ee896a45..268462082 100644 --- a/measure/src/test/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgoTest.scala +++ b/measure/src/test/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgoTest.scala @@ -1,261 +1,261 @@ -/* -Licensed to the Apache Software Foundation (ASF) under one -or more contributor license agreements. See the NOTICE file -distributed with this work for additional information -regarding copyright ownership. The ASF licenses this file -to you under the Apache License, Version 2.0 (the -"License"); you may not use this file except in compliance -with the License. You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, -software distributed under the License is distributed on an -"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -KIND, either express or implied. See the License for the -specific language governing permissions and limitations -under the License. -*/ -package org.apache.griffin.measure.algo.streaming - -import java.util.Date -import java.util.concurrent.TimeUnit - -import org.apache.griffin.measure.algo.batch.BatchAccuracyAlgo -import org.apache.griffin.measure.cache.info.{InfoCacheInstance, TimeInfoCache} -import org.apache.griffin.measure.cache.result._ -import org.apache.griffin.measure.config.params._ -import org.apache.griffin.measure.config.params.env._ -import org.apache.griffin.measure.config.params.user._ -import org.apache.griffin.measure.config.reader._ -import org.apache.griffin.measure.config.validator._ -import org.apache.griffin.measure.connector.direct.DirectDataConnector -import org.apache.griffin.measure.connector.{DataConnector, DataConnectorFactory} -import org.apache.griffin.measure.log.Loggable -import org.apache.griffin.measure.persist.{Persist, PersistFactory, PersistType} -import org.apache.griffin.measure.result._ -import org.apache.griffin.measure.rule.expr._ -import org.apache.griffin.measure.rule.{ExprValueUtil, RuleAnalyzer, RuleFactory} -import org.apache.griffin.measure.utils.{HdfsUtil, TimeUtil} -import org.apache.spark.rdd.RDD -import org.apache.spark.sql.SQLContext -import org.apache.spark.sql.hive.HiveContext -import org.apache.spark.streaming.{Milliseconds, StreamingContext} -import org.apache.spark.{SparkConf, SparkContext} -import org.junit.runner.RunWith -import org.scalatest.junit.JUnitRunner -import org.scalatest.{BeforeAndAfter, FunSuite, Matchers} - -import scala.util.{Failure, Success, Try} - - -@RunWith(classOf[JUnitRunner]) -class StreamingAccuracyAlgoTest extends FunSuite with Matchers with BeforeAndAfter with Loggable { - - val envFile = "src/test/resources/env-streaming.json" - val confFile = "src/test/resources/config-streaming3.json" - val envFsType = "local" - val userFsType = "local" - - val args = Array(envFile, confFile) - - var sc: SparkContext = _ - var sqlContext: SQLContext = _ -// val ssc: StreamingContext = _ - - var allParam: AllParam = _ - - before { - // read param files - val envParam = readParamFile[EnvParam](envFile, envFsType) match { - case Success(p) => p - case Failure(ex) => { - error(ex.getMessage) - sys.exit(-2) - } - } - val userParam = readParamFile[UserParam](confFile, userFsType) match { - case Success(p) => p - case Failure(ex) => { - error(ex.getMessage) - sys.exit(-2) - } - } - allParam = AllParam(envParam, userParam) - - // validate param files - validateParams(allParam) match { - case Failure(ex) => { - error(ex.getMessage) - sys.exit(-3) - } - case _ => { - info("params validation pass") - } - } - - val metricName = userParam.name - val sparkParam = envParam.sparkParam - val conf = new SparkConf().setMaster("local[*]").setAppName(metricName) - conf.setAll(sparkParam.config) - sc = new SparkContext(conf) - sc.setLogLevel(envParam.sparkParam.logLevel) - sqlContext = new SQLContext(sc) -// sqlContext = new HiveContext(sc) - -// val a = sqlContext.sql("select * from s1 limit 10") -// // val a = sqlContext.sql("show tables") -// a.show(10) -// -// val b = HdfsUtil.existPath("/griffin/streaming") -// println(b) - } - - test("algorithm") { - val envParam = allParam.envParam - val userParam = allParam.userParam - val metricName = userParam.name - val sparkParam = envParam.sparkParam - val cleanerParam = envParam.cleanerParam - -// val batchInterval = TimeUtil.milliseconds(sparkParam.batchInterval) match { -// case Some(interval) => Milliseconds(interval) -// case _ => throw new Exception("invalid batch interval") +///* +//Licensed to the Apache Software Foundation (ASF) under one +//or more contributor license agreements. See the NOTICE file +//distributed with this work for additional information +//regarding copyright ownership. The ASF licenses this file +//to you under the Apache License, Version 2.0 (the +//"License"); you may not use this file except in compliance +//with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +//Unless required by applicable law or agreed to in writing, +//software distributed under the License is distributed on an +//"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +//KIND, either express or implied. See the License for the +//specific language governing permissions and limitations +//under the License. +//*/ +//package org.apache.griffin.measure.algo.streaming +// +//import java.util.Date +//import java.util.concurrent.TimeUnit +// +//import org.apache.griffin.measure.algo.batch.BatchAccuracyAlgo +//import org.apache.griffin.measure.cache.info.{InfoCacheInstance, TimeInfoCache} +//import org.apache.griffin.measure.cache.result._ +//import org.apache.griffin.measure.config.params._ +//import org.apache.griffin.measure.config.params.env._ +//import org.apache.griffin.measure.config.params.user._ +//import org.apache.griffin.measure.config.reader._ +//import org.apache.griffin.measure.config.validator._ +//import org.apache.griffin.measure.connector.direct.DirectDataConnector +//import org.apache.griffin.measure.connector.{DataConnector, DataConnectorFactory} +//import org.apache.griffin.measure.log.Loggable +//import org.apache.griffin.measure.persist.{Persist, PersistFactory, PersistType} +//import org.apache.griffin.measure.result._ +//import org.apache.griffin.measure.rule.expr._ +//import org.apache.griffin.measure.rule.{ExprValueUtil, RuleAnalyzer, RuleFactory} +//import org.apache.griffin.measure.utils.{HdfsUtil, TimeUtil} +//import org.apache.spark.rdd.RDD +//import org.apache.spark.sql.SQLContext +//import org.apache.spark.sql.hive.HiveContext +//import org.apache.spark.streaming.{Milliseconds, StreamingContext} +//import org.apache.spark.{SparkConf, SparkContext} +//import org.junit.runner.RunWith +//import org.scalatest.junit.JUnitRunner +//import org.scalatest.{BeforeAndAfter, FunSuite, Matchers} +// +//import scala.util.{Failure, Success, Try} +// +// +//@RunWith(classOf[JUnitRunner]) +//class StreamingAccuracyAlgoTest extends FunSuite with Matchers with BeforeAndAfter with Loggable { +// +// val envFile = "src/test/resources/env-streaming.json" +// val confFile = "src/test/resources/config-streaming3.json" +// val envFsType = "local" +// val userFsType = "local" +// +// val args = Array(envFile, confFile) +// +// var sc: SparkContext = _ +// var sqlContext: SQLContext = _ +//// val ssc: StreamingContext = _ +// +// var allParam: AllParam = _ +// +// before { +// // read param files +// val envParam = readParamFile[EnvParam](envFile, envFsType) match { +// case Success(p) => p +// case Failure(ex) => { +// error(ex.getMessage) +// sys.exit(-2) +// } // } -// val ssc = new StreamingContext(sc, batchInterval) -// ssc.checkpoint(sparkParam.cpDir) - - def createStreamingContext(): StreamingContext = { - val batchInterval = TimeUtil.milliseconds(sparkParam.batchInterval) match { - case Some(interval) => Milliseconds(interval) - case _ => throw new Exception("invalid batch interval") - } - val ssc = new StreamingContext(sc, batchInterval) - ssc.checkpoint(sparkParam.cpDir) - ssc - } - val ssc = StreamingContext.getOrCreate(sparkParam.cpDir, createStreamingContext) - - // start time - val startTime = new Date().getTime() - - val persistFactory = PersistFactory(envParam.persistParams, metricName) - - // get persists to persist measure result - val appPersist: Persist = persistFactory.getPersists(startTime) - - // get spark application id - val applicationId = sc.applicationId - - // persist start id - appPersist.start(applicationId) - - InfoCacheInstance.initInstance(envParam.infoCacheParams, metricName) - InfoCacheInstance.init - - // generate rule from rule param, generate rule analyzer - val ruleFactory = RuleFactory(userParam.evaluateRuleParam) - val rule: StatementExpr = ruleFactory.generateRule() - val ruleAnalyzer: RuleAnalyzer = RuleAnalyzer(rule) - - // const expr value map - val constExprValueMap = ExprValueUtil.genExprValueMaps(None, ruleAnalyzer.constCacheExprs, Map[String, Any]()) - val finalConstExprValueMap = ExprValueUtil.updateExprValueMaps(ruleAnalyzer.constFinalCacheExprs, constExprValueMap) - val finalConstMap = finalConstExprValueMap.headOption match { - case Some(m) => m - case _ => Map[String, Any]() - } - - // data connector - val sourceDataConnector: DirectDataConnector = - DataConnectorFactory.getDirectDataConnector(sqlContext, ssc, userParam.sourceParam, - ruleAnalyzer.sourceRuleExprs, finalConstMap - ) match { - case Success(cntr) => { - if (cntr.available) cntr - else throw new Exception("source data connection error!") - } - case Failure(ex) => throw ex - } - val targetDataConnector: DirectDataConnector = - DataConnectorFactory.getDirectDataConnector(sqlContext, ssc, userParam.targetParam, - ruleAnalyzer.targetRuleExprs, finalConstMap - ) match { - case Success(cntr) => { - if (cntr.available) cntr - else throw new Exception("target data connection error!") - } - case Failure(ex) => throw ex - } - - val cacheResultProcesser = CacheResultProcesser() - - // init data stream - sourceDataConnector.init() - targetDataConnector.init() - - // my algo - val algo = StreamingAccuracyAlgo(allParam) - - val streamingAccuracyProcess = StreamingAccuracyProcess( - sourceDataConnector, targetDataConnector, - ruleAnalyzer, cacheResultProcesser, persistFactory, appPersist) - - val processInterval = TimeUtil.milliseconds(sparkParam.processInterval) match { - case Some(interval) => interval - case _ => throw new Exception("invalid batch interval") - } - val process = TimingProcess(processInterval, streamingAccuracyProcess) - - // clean thread -// case class Clean() extends Runnable { -// val lock = InfoCacheInstance.genLock("clean") -// def run(): Unit = { -// val locked = lock.lock(5, TimeUnit.SECONDS) -// if (locked) { -// try { -// sourceDataConnector.cleanData -// targetDataConnector.cleanData -// } finally { -// lock.unlock() -// } -// } +// val userParam = readParamFile[UserParam](confFile, userFsType) match { +// case Success(p) => p +// case Failure(ex) => { +// error(ex.getMessage) +// sys.exit(-2) +// } +// } +// allParam = AllParam(envParam, userParam) +// +// // validate param files +// validateParams(allParam) match { +// case Failure(ex) => { +// error(ex.getMessage) +// sys.exit(-3) +// } +// case _ => { +// info("params validation pass") +// } +// } +// +// val metricName = userParam.name +// val sparkParam = envParam.sparkParam +// val conf = new SparkConf().setMaster("local[*]").setAppName(metricName) +// conf.setAll(sparkParam.config) +// sc = new SparkContext(conf) +// sc.setLogLevel(envParam.sparkParam.logLevel) +// sqlContext = new SQLContext(sc) +//// sqlContext = new HiveContext(sc) +// +//// val a = sqlContext.sql("select * from s1 limit 10") +//// // val a = sqlContext.sql("show tables") +//// a.show(10) +//// +//// val b = HdfsUtil.existPath("/griffin/streaming") +//// println(b) +// } +// +// test("algorithm") { +// val envParam = allParam.envParam +// val userParam = allParam.userParam +// val metricName = userParam.name +// val sparkParam = envParam.sparkParam +// val cleanerParam = envParam.cleanerParam +// +//// val batchInterval = TimeUtil.milliseconds(sparkParam.batchInterval) match { +//// case Some(interval) => Milliseconds(interval) +//// case _ => throw new Exception("invalid batch interval") +//// } +//// val ssc = new StreamingContext(sc, batchInterval) +//// ssc.checkpoint(sparkParam.cpDir) +// +// def createStreamingContext(): StreamingContext = { +// val batchInterval = TimeUtil.milliseconds(sparkParam.batchInterval) match { +// case Some(interval) => Milliseconds(interval) +// case _ => throw new Exception("invalid batch interval") // } +// val ssc = new StreamingContext(sc, batchInterval) +// ssc.checkpoint(sparkParam.cpDir) +// ssc +// } +// val ssc = StreamingContext.getOrCreate(sparkParam.cpDir, createStreamingContext) +// +// // start time +// val startTime = new Date().getTime() +// +// val persistFactory = PersistFactory(envParam.persistParams, metricName) +// +// // get persists to persist measure result +// val appPersist: Persist = persistFactory.getPersists(startTime) +// +// // get spark application id +// val applicationId = sc.applicationId +// +// // persist start id +// appPersist.start(applicationId) +// +// InfoCacheInstance.initInstance(envParam.infoCacheParams, metricName) +// InfoCacheInstance.init +// +// // generate rule from rule param, generate rule analyzer +// val ruleFactory = RuleFactory(userParam.evaluateRuleParam) +// val rule: StatementExpr = ruleFactory.generateRule() +// val ruleAnalyzer: RuleAnalyzer = RuleAnalyzer(rule) +// +// // const expr value map +// val constExprValueMap = ExprValueUtil.genExprValueMaps(None, ruleAnalyzer.constCacheExprs, Map[String, Any]()) +// val finalConstExprValueMap = ExprValueUtil.updateExprValueMaps(ruleAnalyzer.constFinalCacheExprs, constExprValueMap) +// val finalConstMap = finalConstExprValueMap.headOption match { +// case Some(m) => m +// case _ => Map[String, Any]() // } -// val cleanInterval = TimeUtil.milliseconds(cleanerParam.cleanInterval) match { +// +// // data connector +// val sourceDataConnector: DirectDataConnector = +// DataConnectorFactory.getDirectDataConnector(sqlContext, ssc, userParam.sourceParam, +// ruleAnalyzer.sourceRuleExprs, finalConstMap +// ) match { +// case Success(cntr) => { +// if (cntr.available) cntr +// else throw new Exception("source data connection error!") +// } +// case Failure(ex) => throw ex +// } +// val targetDataConnector: DirectDataConnector = +// DataConnectorFactory.getDirectDataConnector(sqlContext, ssc, userParam.targetParam, +// ruleAnalyzer.targetRuleExprs, finalConstMap +// ) match { +// case Success(cntr) => { +// if (cntr.available) cntr +// else throw new Exception("target data connection error!") +// } +// case Failure(ex) => throw ex +// } +// +// val cacheResultProcesser = CacheResultProcesser() +// +// // init data stream +// sourceDataConnector.init() +// targetDataConnector.init() +// +// // my algo +// val algo = StreamingAccuracyAlgo(allParam) +// +// val streamingAccuracyProcess = StreamingAccuracyProcess( +// sourceDataConnector, targetDataConnector, +// ruleAnalyzer, cacheResultProcesser, persistFactory, appPersist) +// +// val processInterval = TimeUtil.milliseconds(sparkParam.processInterval) match { // case Some(interval) => interval // case _ => throw new Exception("invalid batch interval") // } -// val clean = TimingProcess(cleanInterval, Clean()) - - process.startup() -// clean.startup() - - ssc.start() - ssc.awaitTermination() - ssc.stop(stopSparkContext=true, stopGracefully=true) - - println("================ end ================") - - // context stop - sc.stop - - InfoCacheInstance.close - - appPersist.finish() - - process.shutdown() -// clean.shutdown() - } - - private def readParamFile[T <: Param](file: String, fsType: String)(implicit m : Manifest[T]): Try[T] = { - val paramReader = ParamReaderFactory.getParamReader(file, fsType) - paramReader.readConfig[T] - } - - private def validateParams(allParam: AllParam): Try[Boolean] = { - val allParamValidator = AllParamValidator() - allParamValidator.validate(allParam) - } - -} +// val process = TimingProcess(processInterval, streamingAccuracyProcess) +// +// // clean thread +//// case class Clean() extends Runnable { +//// val lock = InfoCacheInstance.genLock("clean") +//// def run(): Unit = { +//// val locked = lock.lock(5, TimeUnit.SECONDS) +//// if (locked) { +//// try { +//// sourceDataConnector.cleanData +//// targetDataConnector.cleanData +//// } finally { +//// lock.unlock() +//// } +//// } +//// } +//// } +//// val cleanInterval = TimeUtil.milliseconds(cleanerParam.cleanInterval) match { +//// case Some(interval) => interval +//// case _ => throw new Exception("invalid batch interval") +//// } +//// val clean = TimingProcess(cleanInterval, Clean()) +// +// process.startup() +//// clean.startup() +// +// ssc.start() +// ssc.awaitTermination() +// ssc.stop(stopSparkContext=true, stopGracefully=true) +// +// println("================ end ================") +// +// // context stop +// sc.stop +// +// InfoCacheInstance.close +// +// appPersist.finish() +// +// process.shutdown() +//// clean.shutdown() +// } +// +// private def readParamFile[T <: Param](file: String, fsType: String)(implicit m : Manifest[T]): Try[T] = { +// val paramReader = ParamReaderFactory.getParamReader(file, fsType) +// paramReader.readConfig[T] +// } +// +// private def validateParams(allParam: AllParam): Try[Boolean] = { +// val allParamValidator = AllParamValidator() +// allParamValidator.validate(allParam) +// } +// +//} From c70ca1d733c14365b6fc06db6db2fbcd5962b953 Mon Sep 17 00:00:00 2001 From: Lionel Liu Date: Tue, 15 Aug 2017 10:21:50 +0800 Subject: [PATCH 048/111] t1 --- measure/src/test/resources/config.json | 2 +- .../algo/batch/BatchAccuracyAlgoTest.scala | 475 +++++++++++------- 2 files changed, 284 insertions(+), 193 deletions(-) diff --git a/measure/src/test/resources/config.json b/measure/src/test/resources/config.json index 08a60212e..0a174744a 100644 --- a/measure/src/test/resources/config.json +++ b/measure/src/test/resources/config.json @@ -22,6 +22,6 @@ "evaluateRule": { "sampleRatio": 1, - "rules": "$source.user_id + 5 = $target.user_id + (2 + 3) AND $source.first_name + 12 = $target.first_name + (10 + 2) 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 WHEN $source.user_id > 10015" + "rules": "$source.user_id = $target.user_id AND $source.first_name = $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" } } \ No newline at end of file diff --git a/measure/src/test/scala/org/apache/griffin/measure/algo/batch/BatchAccuracyAlgoTest.scala b/measure/src/test/scala/org/apache/griffin/measure/algo/batch/BatchAccuracyAlgoTest.scala index 6a603266f..360c7fac9 100644 --- a/measure/src/test/scala/org/apache/griffin/measure/algo/batch/BatchAccuracyAlgoTest.scala +++ b/measure/src/test/scala/org/apache/griffin/measure/algo/batch/BatchAccuracyAlgoTest.scala @@ -1,198 +1,289 @@ -///* -//Licensed to the Apache Software Foundation (ASF) under one -//or more contributor license agreements. See the NOTICE file -//distributed with this work for additional information -//regarding copyright ownership. The ASF licenses this file -//to you under the Apache License, Version 2.0 (the -//"License"); you may not use this file except in compliance -//with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -//Unless required by applicable law or agreed to in writing, -//software distributed under the License is distributed on an -//"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -//KIND, either express or implied. See the License for the -//specific language governing permissions and limitations -//under the License. -//*/ -//package org.apache.griffin.measure.algo -// -//import java.util.Date -// -//import org.apache.griffin.measure.algo.batch.BatchAccuracyAlgo -//import org.apache.griffin.measure.config.params._ -//import org.apache.griffin.measure.config.params.env._ -//import org.apache.griffin.measure.config.params.user._ -//import org.apache.griffin.measure.config.reader._ -//import org.apache.griffin.measure.config.validator._ -//import org.apache.griffin.measure.connector.direct.DirectDataConnector -//import org.apache.griffin.measure.connector.{DataConnector, DataConnectorFactory} -//import org.apache.griffin.measure.log.Loggable -//import org.apache.griffin.measure.rule.expr._ -//import org.apache.griffin.measure.rule.{ExprValueUtil, RuleAnalyzer, RuleFactory} -//import org.apache.spark.rdd.RDD -//import org.apache.spark.sql.SQLContext -//import org.apache.spark.{SparkConf, SparkContext} -//import org.junit.runner.RunWith -//import org.scalatest.junit.JUnitRunner -//import org.scalatest.{BeforeAndAfter, FunSuite, Matchers} -// -//import scala.util.{Failure, Success, Try} -// -// -//@RunWith(classOf[JUnitRunner]) -//class BatchAccuracyAlgoTest extends FunSuite with Matchers with BeforeAndAfter with Loggable { -// -// val envFile = "src/test/resources/env.json" -// val confFile = "src/test/resources/config.json" -//// val confFile = "{\"name\":\"accu1\",\"type\":\"accuracy\",\"source\":{\"type\":\"avro\",\"version\":\"1.7\",\"config\":{\"file.name\":\"src/test/resources/users_info_src.avro\"}},\"target\":{\"type\":\"avro\",\"version\":\"1.7\",\"config\":{\"file.name\":\"src/test/resources/users_info_target.avro\"}},\"evaluateRule\":{\"sampleRatio\":1,\"rules\":\"$source.user_id + 5 = $target.user_id + (2 + 3) AND $source.first_name + 12 = $target.first_name + (10 + 2) 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 AND (15 OR true) WHEN true AND $source.user_id > 10020\"}}" -// val envFsType = "local" -// val userFsType = "local" -// -// val args = Array(envFile, confFile) -// -// var sc: SparkContext = _ -// var sqlContext: SQLContext = _ -// -// var allParam: AllParam = _ -// -// before { -// // read param files -// val envParam = readParamFile[EnvParam](envFile, envFsType) match { -// case Success(p) => p -// case Failure(ex) => { -// error(ex.getMessage) -// sys.exit(-2) -// } -// } -// val userParam = readParamFile[UserParam](confFile, userFsType) match { -// case Success(p) => p -// case Failure(ex) => { -// error(ex.getMessage) -// sys.exit(-2) -// } -// } -// allParam = AllParam(envParam, userParam) -// -// // validate param files -// validateParams(allParam) match { -// case Failure(ex) => { -// error(ex.getMessage) -// sys.exit(-3) -// } -// case _ => { -// info("params validation pass") -// } -// } -// -// val metricName = userParam.name -// val conf = new SparkConf().setMaster("local[*]").setAppName(metricName) -// sc = new SparkContext(conf) +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ +package org.apache.griffin.measure.algo + +import java.util.Date + +import org.apache.griffin.measure.algo.batch.BatchAccuracyAlgo +import org.apache.griffin.measure.config.params._ +import org.apache.griffin.measure.config.params.env._ +import org.apache.griffin.measure.config.params.user._ +import org.apache.griffin.measure.config.reader._ +import org.apache.griffin.measure.config.validator._ +import org.apache.griffin.measure.connector.direct.DirectDataConnector +import org.apache.griffin.measure.connector.{DataConnector, DataConnectorFactory} +import org.apache.griffin.measure.log.Loggable +import org.apache.griffin.measure.rule.expr._ +import org.apache.griffin.measure.rule.{ExprValueUtil, RuleAnalyzer, RuleFactory} +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.hive.HiveContext +import org.apache.spark.{SparkConf, SparkContext} +import org.junit.runner.RunWith +import org.scalatest.junit.JUnitRunner +import org.scalatest.{BeforeAndAfter, FunSuite, Matchers} + +import scala.util.{Failure, Success, Try} + + +@RunWith(classOf[JUnitRunner]) +class BatchAccuracyAlgoTest extends FunSuite with Matchers with BeforeAndAfter with Loggable { + + val envFile = "src/test/resources/env.json" + val confFile = "src/test/resources/config.json" +// val confFile = "{\"name\":\"accu1\",\"type\":\"accuracy\",\"source\":{\"type\":\"avro\",\"version\":\"1.7\",\"config\":{\"file.name\":\"src/test/resources/users_info_src.avro\"}},\"target\":{\"type\":\"avro\",\"version\":\"1.7\",\"config\":{\"file.name\":\"src/test/resources/users_info_target.avro\"}},\"evaluateRule\":{\"sampleRatio\":1,\"rules\":\"$source.user_id + 5 = $target.user_id + (2 + 3) AND $source.first_name + 12 = $target.first_name + (10 + 2) 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 AND (15 OR true) WHEN true AND $source.user_id > 10020\"}}" + val envFsType = "local" + val userFsType = "local" + + val args = Array(envFile, confFile) + + var sc: SparkContext = _ + var sqlContext: SQLContext = _ + + var allParam: AllParam = _ + + before { + // read param files + val envParam = readParamFile[EnvParam](envFile, envFsType) match { + case Success(p) => p + case Failure(ex) => { + error(ex.getMessage) + sys.exit(-2) + } + } + val userParam = readParamFile[UserParam](confFile, userFsType) match { + case Success(p) => p + case Failure(ex) => { + error(ex.getMessage) + sys.exit(-2) + } + } + allParam = AllParam(envParam, userParam) + + // validate param files + validateParams(allParam) match { + case Failure(ex) => { + error(ex.getMessage) + sys.exit(-3) + } + case _ => { + info("params validation pass") + } + } + + val metricName = userParam.name + val conf = new SparkConf().setMaster("local[*]").setAppName(metricName) + sc = new SparkContext(conf) // sqlContext = new SQLContext(sc) -// } -// -// test("algorithm") { -// Try { -// val envParam = allParam.envParam -// val userParam = allParam.userParam -// -// // start time -// val startTime = new Date().getTime() -// -// // get spark application id -// val applicationId = sc.applicationId -// -// // rules -// val ruleFactory = RuleFactory(userParam.evaluateRuleParam) -// val rule: StatementExpr = ruleFactory.generateRule() -// val ruleAnalyzer: RuleAnalyzer = RuleAnalyzer(rule) -// -// ruleAnalyzer.constCacheExprs.foreach(println) -// ruleAnalyzer.constFinalCacheExprs.foreach(println) -// -// // global cache data -// val constExprValueMap = ExprValueUtil.genExprValueMaps(None, ruleAnalyzer.constCacheExprs, Map[String, Any]()) -// val finalConstExprValueMap = ExprValueUtil.updateExprValueMaps(ruleAnalyzer.constFinalCacheExprs, constExprValueMap) -// val finalConstMap = finalConstExprValueMap.headOption match { -// case Some(m) => m -// case _ => Map[String, Any]() -// } -// -// // data connector -// val sourceDataConnector: DirectDataConnector = -// DataConnectorFactory.getDirectDataConnector(sqlContext, null, userParam.sourceParam, -// ruleAnalyzer.sourceRuleExprs, finalConstMap -// ) match { -// case Success(cntr) => { -// if (cntr.available) cntr -// else throw new Exception("source data not available!") -// } -// case Failure(ex) => throw ex -// } -// val targetDataConnector: DirectDataConnector = -// DataConnectorFactory.getDirectDataConnector(sqlContext, null, userParam.targetParam, -// ruleAnalyzer.targetRuleExprs, finalConstMap -// ) match { -// case Success(cntr) => { -// if (cntr.available) cntr -// else throw new Exception("target data not available!") -// } -// case Failure(ex) => throw ex -// } -// -// // get metadata -//// val sourceMetaData: Iterable[(String, String)] = sourceDataConnector.metaData() match { -//// case Success(md) => md -//// case Failure(ex) => throw ex -//// } -//// val targetMetaData: Iterable[(String, String)] = targetDataConnector.metaData() match { -//// case Success(md) => md -//// case Failure(ex) => throw ex -//// } -// -// // get data -// val sourceData: RDD[(Product, (Map[String, Any], Map[String, Any]))] = sourceDataConnector.data() match { -// case Success(dt) => dt + sqlContext = new HiveContext(sc) + } + + test("algorithm") { + Try { + val envParam = allParam.envParam + val userParam = allParam.userParam + + // start time + val startTime = new Date().getTime() + + // get spark application id + val applicationId = sc.applicationId + + // rules + val ruleFactory = RuleFactory(userParam.evaluateRuleParam) + val rule: StatementExpr = ruleFactory.generateRule() + val ruleAnalyzer: RuleAnalyzer = RuleAnalyzer(rule) + + ruleAnalyzer.constCacheExprs.foreach(println) + ruleAnalyzer.constFinalCacheExprs.foreach(println) + + // global cache data + val constExprValueMap = ExprValueUtil.genExprValueMaps(None, ruleAnalyzer.constCacheExprs, Map[String, Any]()) + val finalConstExprValueMap = ExprValueUtil.updateExprValueMaps(ruleAnalyzer.constFinalCacheExprs, constExprValueMap) + val finalConstMap = finalConstExprValueMap.headOption match { + case Some(m) => m + case _ => Map[String, Any]() + } + + // data connector + val sourceDataConnector: DirectDataConnector = + DataConnectorFactory.getDirectDataConnector(sqlContext, null, userParam.sourceParam, + ruleAnalyzer.sourceRuleExprs, finalConstMap + ) match { + case Success(cntr) => { + if (cntr.available) cntr + else throw new Exception("source data not available!") + } + case Failure(ex) => throw ex + } + val targetDataConnector: DirectDataConnector = + DataConnectorFactory.getDirectDataConnector(sqlContext, null, userParam.targetParam, + ruleAnalyzer.targetRuleExprs, finalConstMap + ) match { + case Success(cntr) => { + if (cntr.available) cntr + else throw new Exception("target data not available!") + } + case Failure(ex) => throw ex + } + + // get metadata +// val sourceMetaData: Iterable[(String, String)] = sourceDataConnector.metaData() match { +// case Success(md) => md // case Failure(ex) => throw ex // } -// val targetData: RDD[(Product, (Map[String, Any], Map[String, Any]))] = targetDataConnector.data() match { -// case Success(dt) => dt +// val targetMetaData: Iterable[(String, String)] = targetDataConnector.metaData() match { +// case Success(md) => md // case Failure(ex) => throw ex // } + + // get data + val sourceData: RDD[(Product, (Map[String, Any], Map[String, Any]))] = sourceDataConnector.data() match { + case Success(dt) => dt + case Failure(ex) => throw ex + } + val targetData: RDD[(Product, (Map[String, Any], Map[String, Any]))] = targetDataConnector.data() match { + case Success(dt) => dt + case Failure(ex) => throw ex + } + + // my algo + val algo = BatchAccuracyAlgo(allParam) + + // accuracy algorithm + val (accuResult, missingRdd, matchedRdd) = algo.accuracy(sourceData, targetData, ruleAnalyzer) + + println(s"match percentage: ${accuResult.matchPercentage}, total count: ${accuResult.total}") + + missingRdd.map(rec => algo.record2String(rec, ruleAnalyzer.sourceRuleExprs.persistExprs, ruleAnalyzer.targetRuleExprs.persistExprs)).foreach(println) + + // end time + val endTime = new Date().getTime + println(s"using time: ${endTime - startTime} ms") + } match { + case Failure(ex) => { + error(ex.getMessage) + sys.exit(-4) + } + case _ => { + info("calculation finished") + } + } + } + + private def readParamFile[T <: Param](file: String, fsType: String)(implicit m : Manifest[T]): Try[T] = { + val paramReader = ParamReaderFactory.getParamReader(file, fsType) + paramReader.readConfig[T] + } + + private def validateParams(allParam: AllParam): Try[Boolean] = { + val allParamValidator = AllParamValidator() + allParamValidator.validate(allParam) + } + + test ("spark sql") { + Try { + val envParam = allParam.envParam + val userParam = allParam.userParam + + // start time + val startTime = new Date().getTime() + + // get spark application id + val applicationId = sc.applicationId + +// val sourceFilePath = "src/test/resources/users_info_src.avro" +// val targetFilePath = "src/test/resources/users_info_target.avro" // -// // my algo -// val algo = BatchAccuracyAlgo(allParam) -// -// // accuracy algorithm -// val (accuResult, missingRdd, matchedRdd) = algo.accuracy(sourceData, targetData, ruleAnalyzer) -// -// println(s"match percentage: ${accuResult.matchPercentage}, total count: ${accuResult.total}") -// -// missingRdd.map(rec => algo.record2String(rec, ruleAnalyzer.sourceRuleExprs.persistExprs, ruleAnalyzer.targetRuleExprs.persistExprs)).foreach(println) -// -// // end time -// val endTime = new Date().getTime -// println(s"using time: ${endTime - startTime} ms") -// } match { -// case Failure(ex) => { -// error(ex.getMessage) -// sys.exit(-4) -// } -// case _ => { -// info("calculation finished") -// } -// } -// } -// -// private def readParamFile[T <: Param](file: String, fsType: String)(implicit m : Manifest[T]): Try[T] = { -// val paramReader = ParamReaderFactory.getParamReader(file, fsType) -// paramReader.readConfig[T] -// } -// -// private def validateParams(allParam: AllParam): Try[Boolean] = { -// val allParamValidator = AllParamValidator() -// allParamValidator.validate(allParam) -// } -// -//} +// val sourceDF = sqlContext.read.format("com.databricks.spark.avro").load(sourceFilePath) +// val targetDF = sqlContext.read.format("com.databricks.spark.avro").load(targetFilePath) + + val sourceTableName = "source" + val targetTableName = "target" + + val sourceDF = sqlContext.sql("SELECT * FROM default.data_rdm") + val targetDF = sqlContext.sql("SELECT * FROM default.data_rdm") + +// sourceDF.show(100) +// targetDF.show(100) + + sourceDF.registerTempTable(sourceTableName) + targetDF.registerTempTable(targetTableName) + + val sql = + s""" + |SELECT COUNT(*) FROM ${sourceTableName} LEFT JOIN ${targetTableName} + |ON ${sourceTableName}.uid = ${targetTableName}.uid + """.stripMargin + +// val sql = +// """ +// |SELECT COUNT(*) FROM source LEFT JOIN target +// |ON coalesce(source.user_id, 'null') = coalesce(target.user_id, 'null') +// |AND coalesce(source.first_name, 'null') = coalesce(target.first_name, 'null') +// |AND coalesce(source.last_name, 'null') = coalesce(target.last_name, 'null') +// |AND coalesce(source.address, 'null') = coalesce(target.address, 'null') +// |AND coalesce(source.email, 'null') = coalesce(target.email, 'null') +// |AND coalesce(source.phone, 'null') = coalesce(target.phone, 'null') +// |AND coalesce(source.post_code, 'null') = coalesce(target.post_code, 'null') +// |WHERE (target.user_id IS NULL +// |AND target.first_name IS NULL +// |AND target.last_name IS NULL +// |AND target.address IS NULL +// |AND target.email IS NULL +// |AND target.phone IS NULL +// |AND target.post_code IS NULL) +// """.stripMargin + +// val sql = +// """ +// |SELECT * FROM source LEFT JOIN target +// |ON source.user_id = target.user_id +// |AND source.first_name = 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 coalesce(source.post_code, 'null') = coalesce(target.post_code, 'null') +// """.stripMargin + +// val sql = +// """ +// |SELECT * FROM source WHERE source.post_code IS NULL +// """.stripMargin + + val result = sqlContext.sql(sql) + + result.show(100) + + // end time + val endTime = new Date().getTime + println(s"using time: ${endTime - startTime} ms") + } match { + case Failure(ex) => { + error(ex.getMessage) + sys.exit(-4) + } + case _ => { + info("calculation finished") + } + } + } + +} From 7936374fe26b750c1adc5f0cc2211210b1c3aa45 Mon Sep 17 00:00:00 2001 From: Lionel Liu Date: Tue, 15 Aug 2017 15:55:46 +0800 Subject: [PATCH 049/111] t2 --- measure/src/main/resources/config-old.json | 31 +++++++++++ measure/src/main/resources/config-sql.json | 53 ++++++++++++++++++ measure/src/main/resources/config.json | 55 ++++++++++++------- .../algo/batch/BatchAccuracyAlgoTest.scala | 55 ++++++++++--------- 4 files changed, 148 insertions(+), 46 deletions(-) create mode 100644 measure/src/main/resources/config-old.json create mode 100644 measure/src/main/resources/config-sql.json diff --git a/measure/src/main/resources/config-old.json b/measure/src/main/resources/config-old.json new file mode 100644 index 000000000..ab32b75cf --- /dev/null +++ b/measure/src/main/resources/config-old.json @@ -0,0 +1,31 @@ +{ + "name": "accu1", + "type": "accuracy", + + "process.type": "batch", + + "source": { + "type": "hive", + "version": "1.2", + "config": { + "database": "default", + "table.name": "users_info_src", + "partitions": "dt=23123, hour=432; dt=35464, hour=4657" + } + }, + + "target": { + "type": "hive", + "version": "1.2", + "config": { + "database": "default", + "table.name": "users_info_target", + "partitions": "dt=23123, hour=432; dt=35464, hour=4657" + } + }, + + "evaluateRule": { + "sampleRatio": 0.2, + "rules": "$source.user_id = $target.user_id AND $source.first_name = $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" + } +} \ No newline at end of file diff --git a/measure/src/main/resources/config-sql.json b/measure/src/main/resources/config-sql.json new file mode 100644 index 000000000..0db0a6e2c --- /dev/null +++ b/measure/src/main/resources/config-sql.json @@ -0,0 +1,53 @@ +{ + "name": "accu1", + + "process.type": "batch", + + "data.sources": [ + { + "name": "source", + "connectors": [ + { + "type": "hive", + "config": { + "database": "default", + "table.name": "users_info_src", + "partitions": "dt=23123, hour=432; dt=35464, hour=4657" + } + } + ] + }, { + "name": "target", + "connectors": [ + { + "type": "hive", + "version": "1.2", + "config": { + "database": "default", + "table.name": "users_info_target", + "partitions": "dt=23123, hour=432; dt=35464, hour=4657" + } + } + ] + } + ], + + "evaluateRule": { + "dsl.type": "spark-sql", + "rules": [ + { + "name": "miss.record", + "rule": "SELECT source.name FROM source LEFT JOIN target ON coalesce(source.name, 'null') = coalesce(target.name, 'null') WHERE (NOT (source.name IS NULL)) AND (target.name IS NULL)", + "persist.type": "record" + }, { + "name": "miss.count", + "rule": "SELECT COUNT(*) FROM miss", + "persist.type": "metric" + }, { + "name": "total.count", + "rule": "SELECT COUNT(*) FROM source", + "persist.type": "metric" + } + ] + } +} \ No newline at end of file diff --git a/measure/src/main/resources/config.json b/measure/src/main/resources/config.json index ab32b75cf..42c57c3d6 100644 --- a/measure/src/main/resources/config.json +++ b/measure/src/main/resources/config.json @@ -1,31 +1,44 @@ { "name": "accu1", - "type": "accuracy", "process.type": "batch", - "source": { - "type": "hive", - "version": "1.2", - "config": { - "database": "default", - "table.name": "users_info_src", - "partitions": "dt=23123, hour=432; dt=35464, hour=4657" + "data.sources": [ + { + "name": "source", + "connectors": [ + { + "type": "hive", + "config": { + "database": "default", + "table.name": "users_info_src", + "partitions": "dt=23123, hour=432; dt=35464, hour=4657" + } + } + ] + }, { + "name": "target", + "connectors": [ + { + "type": "hive", + "version": "1.2", + "config": { + "database": "default", + "table.name": "users_info_target", + "partitions": "dt=23123, hour=432; dt=35464, hour=4657" + } + } + ] } - }, - - "target": { - "type": "hive", - "version": "1.2", - "config": { - "database": "default", - "table.name": "users_info_target", - "partitions": "dt=23123, hour=432; dt=35464, hour=4657" - } - }, + ], "evaluateRule": { - "sampleRatio": 0.2, - "rules": "$source.user_id = $target.user_id AND $source.first_name = $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" + "dsl.type": "griffin-dsl", + "rules": [ + { + "type": "accuracy", + "rule": "$source.user_id = $target.user_id AND $source.first_name = $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" + } + ] } } \ No newline at end of file diff --git a/measure/src/test/scala/org/apache/griffin/measure/algo/batch/BatchAccuracyAlgoTest.scala b/measure/src/test/scala/org/apache/griffin/measure/algo/batch/BatchAccuracyAlgoTest.scala index 360c7fac9..0d26411dc 100644 --- a/measure/src/test/scala/org/apache/griffin/measure/algo/batch/BatchAccuracyAlgoTest.scala +++ b/measure/src/test/scala/org/apache/griffin/measure/algo/batch/BatchAccuracyAlgoTest.scala @@ -214,11 +214,11 @@ class BatchAccuracyAlgoTest extends FunSuite with Matchers with BeforeAndAfter w // val sourceDF = sqlContext.read.format("com.databricks.spark.avro").load(sourceFilePath) // val targetDF = sqlContext.read.format("com.databricks.spark.avro").load(targetFilePath) - val sourceTableName = "source" - val targetTableName = "target" + val sourceTableName = "source.table" + val targetTableName = "target.table" - val sourceDF = sqlContext.sql("SELECT * FROM default.data_rdm") - val targetDF = sqlContext.sql("SELECT * FROM default.data_rdm") + val sourceDF = sqlContext.sql("SELECT * FROM default.data_avr") + val targetDF = sqlContext.sql("SELECT * FROM default.data_only") // sourceDF.show(100) // targetDF.show(100) @@ -226,31 +226,31 @@ class BatchAccuracyAlgoTest extends FunSuite with Matchers with BeforeAndAfter w sourceDF.registerTempTable(sourceTableName) targetDF.registerTempTable(targetTableName) - val sql = - s""" - |SELECT COUNT(*) FROM ${sourceTableName} LEFT JOIN ${targetTableName} - |ON ${sourceTableName}.uid = ${targetTableName}.uid - """.stripMargin +// val sourceTableName = "data_avr" +// val targetTableName = "data_avr" // val sql = -// """ -// |SELECT COUNT(*) FROM source LEFT JOIN target -// |ON coalesce(source.user_id, 'null') = coalesce(target.user_id, 'null') -// |AND coalesce(source.first_name, 'null') = coalesce(target.first_name, 'null') -// |AND coalesce(source.last_name, 'null') = coalesce(target.last_name, 'null') -// |AND coalesce(source.address, 'null') = coalesce(target.address, 'null') -// |AND coalesce(source.email, 'null') = coalesce(target.email, 'null') -// |AND coalesce(source.phone, 'null') = coalesce(target.phone, 'null') -// |AND coalesce(source.post_code, 'null') = coalesce(target.post_code, 'null') -// |WHERE (target.user_id IS NULL -// |AND target.first_name IS NULL -// |AND target.last_name IS NULL -// |AND target.address IS NULL -// |AND target.email IS NULL -// |AND target.phone IS NULL -// |AND target.post_code IS NULL) +// s""" +// |SELECT COUNT(*) FROM `${sourceTableName}` LEFT JOIN `${targetTableName}` +// |ON `${sourceTableName}`.uid = `${targetTableName}`.uid // """.stripMargin + val sql = + s""" + |SELECT `${sourceTableName}`.uid, `${sourceTableName}`.uage, `${sourceTableName}`.udes, + |`${targetTableName}`.uid, `${targetTableName}`.uage, `${targetTableName}`.udes + |FROM `${sourceTableName}` LEFT JOIN `${targetTableName}` + |ON coalesce(`${sourceTableName}`.uid, 'null') = coalesce(`${targetTableName}`.uid, 'null') + |AND coalesce(`${sourceTableName}`.uage, 'null') = coalesce(`${targetTableName}`.uage, 'null') + |AND coalesce(`${sourceTableName}`.udes, 'null') = coalesce(`${targetTableName}`.udes, 'null') + |WHERE (NOT (`${sourceTableName}`.uid IS NULL + |AND `${sourceTableName}`.uage IS NULL + |AND `${sourceTableName}`.udes IS NULL)) + |AND ((`${targetTableName}`.uid IS NULL + |AND `${targetTableName}`.uage IS NULL + |AND `${targetTableName}`.udes IS NULL)) + """.stripMargin + // val sql = // """ // |SELECT * FROM source LEFT JOIN target @@ -272,6 +272,11 @@ class BatchAccuracyAlgoTest extends FunSuite with Matchers with BeforeAndAfter w result.show(100) +// result.registerTempTable("result") +// val rsql = "SELECT COUNT(*) FROM result" +// val rr = sqlContext.sql(rsql) +// rr.show(100) + // end time val endTime = new Date().getTime println(s"using time: ${endTime - startTime} ms") From cd63de7e5e672be9c2202c54edcf6ab9d264e0f4 Mon Sep 17 00:00:00 2001 From: Lionel Liu Date: Tue, 15 Aug 2017 16:18:02 +0800 Subject: [PATCH 050/111] param done --- measure/src/main/resources/config-sql.json | 1 + measure/src/main/resources/config.json | 1 + .../apache/griffin/measure/Application.scala | 266 +++---- .../algo/batch/BatchAccuracyAlgo.scala | 376 ++++----- .../measure/algo/batch/BatchProfileAlgo.scala | 324 ++++---- .../streaming/StreamingAccuracyAlgo.scala | 720 +++++++++--------- .../config/params/user/DataSourceParam.scala | 30 + .../params/user/EvaluateRuleParam.scala | 4 +- .../config/params/user/RuleParam.scala | 32 + .../config/params/user/UserParam.scala | 10 +- .../griffin/measure/rule/RuleFactory.scala | 104 +-- .../algo/batch/BatchAccuracyAlgoTest.scala | 574 +++++++------- .../measure/algo/core/AccuracyCoreTest.scala | 178 ++--- .../measure/algo/core/ProfileCoreTest.scala | 158 ++-- .../reader/ParamRawStringReaderTest.scala | 3 +- .../measure/rule/ExprValueUtilTest.scala | 172 ++--- .../measure/rule/RuleAnalyzerTest.scala | 120 +-- .../measure/rule/RuleFactoryTest.scala | 88 +-- .../griffin/measure/rule/RuleParserTest.scala | 426 +++++------ .../apache/griffin/measure/sql/SqlTest.scala | 234 +++--- .../griffin/measure/utils/JsonUtilTest.scala | 120 +-- 21 files changed, 2001 insertions(+), 1940 deletions(-) create mode 100644 measure/src/main/scala/org/apache/griffin/measure/config/params/user/DataSourceParam.scala create mode 100644 measure/src/main/scala/org/apache/griffin/measure/config/params/user/RuleParam.scala diff --git a/measure/src/main/resources/config-sql.json b/measure/src/main/resources/config-sql.json index 0db0a6e2c..aad95845e 100644 --- a/measure/src/main/resources/config-sql.json +++ b/measure/src/main/resources/config-sql.json @@ -9,6 +9,7 @@ "connectors": [ { "type": "hive", + "version": "1.2", "config": { "database": "default", "table.name": "users_info_src", diff --git a/measure/src/main/resources/config.json b/measure/src/main/resources/config.json index 42c57c3d6..502535b64 100644 --- a/measure/src/main/resources/config.json +++ b/measure/src/main/resources/config.json @@ -9,6 +9,7 @@ "connectors": [ { "type": "hive", + "version": "1.2", "config": { "database": "default", "table.name": "users_info_src", diff --git a/measure/src/main/scala/org/apache/griffin/measure/Application.scala b/measure/src/main/scala/org/apache/griffin/measure/Application.scala index af8c83065..4c322d3b9 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/Application.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/Application.scala @@ -1,133 +1,133 @@ -/* -Licensed to the Apache Software Foundation (ASF) under one -or more contributor license agreements. See the NOTICE file -distributed with this work for additional information -regarding copyright ownership. The ASF licenses this file -to you under the Apache License, Version 2.0 (the -"License"); you may not use this file except in compliance -with the License. You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, -software distributed under the License is distributed on an -"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -KIND, either express or implied. See the License for the -specific language governing permissions and limitations -under the License. -*/ -package org.apache.griffin.measure - -import org.apache.griffin.measure.algo._ -import org.apache.griffin.measure.algo.batch._ -import org.apache.griffin.measure.algo.streaming._ -import org.apache.griffin.measure.config.params._ -import org.apache.griffin.measure.config.params.env._ -import org.apache.griffin.measure.config.params.user._ -import org.apache.griffin.measure.config.reader._ -import org.apache.griffin.measure.config.validator.AllParamValidator -import org.apache.griffin.measure.log.Loggable -import org.apache.griffin.measure.persist.PersistThreadPool - -import scala.util.{Failure, Success, Try} - -object Application extends Loggable { - - def main(args: Array[String]): Unit = { - info(args.toString) - if (args.length < 2) { - error("Usage: class [List of String split by comma: raw | local | hdfs(default)]") - sys.exit(-1) - } - - val envParamFile = args(0) - val userParamFile = args(1) - val (envFsType, userFsType) = if (args.length > 2) { - val fsTypes = args(2).trim.split(",") - if (fsTypes.length == 1) (fsTypes(0).trim, fsTypes(0).trim) - else if (fsTypes.length >= 2) (fsTypes(0).trim, fsTypes(1).trim) - else ("hdfs", "hdfs") - } else ("hdfs", "hdfs") - - info(envParamFile) - info(userParamFile) - - // read param files - val envParam = readParamFile[EnvParam](envParamFile, envFsType) match { - case Success(p) => p - case Failure(ex) => { - error(ex.getMessage) - sys.exit(-2) - } - } - val userParam = readParamFile[UserParam](userParamFile, userFsType) match { - case Success(p) => p - case Failure(ex) => { - error(ex.getMessage) - sys.exit(-2) - } - } - val allParam: AllParam = AllParam(envParam, userParam) - - // validate param files - validateParams(allParam) match { - case Failure(ex) => { - error(ex.getMessage) - sys.exit(-3) - } - case _ => { - info("params validation pass") - } - } - - // choose algorithm - val dqType = allParam.userParam.dqType - val procType = allParam.userParam.procType - val algo: Algo = (dqType, procType) match { - case (MeasureType.accuracy(), ProcessType.batch()) => BatchAccuracyAlgo(allParam) - case (MeasureType.profile(), ProcessType.batch()) => BatchProfileAlgo(allParam) - case (MeasureType.accuracy(), ProcessType.streaming()) => StreamingAccuracyAlgo(allParam) -// case (MeasureType.profile(), ProcessType.streaming()) => StreamingProfileAlgo(allParam) - case _ => { - error(s"${dqType} with ${procType} is unsupported dq type!") - sys.exit(-4) - } - } - - // algorithm run - algo.run match { - case Failure(ex) => { - error(s"app error: ${ex.getMessage}") - - procType match { - case ProcessType.streaming() => { - // streaming need to attempt more times by spark streaming itself - throw ex - } - case _ => { - shutdown - sys.exit(-5) - } - } - } - case _ => { - info("app finished and success") - } - } - } - - private def readParamFile[T <: Param](file: String, fsType: String)(implicit m : Manifest[T]): Try[T] = { - val paramReader = ParamReaderFactory.getParamReader(file, fsType) - paramReader.readConfig[T] - } - - private def validateParams(allParam: AllParam): Try[Boolean] = { - val allParamValidator = AllParamValidator() - allParamValidator.validate(allParam) - } - - private def shutdown(): Unit = { - PersistThreadPool.shutdown - } - -} +///* +//Licensed to the Apache Software Foundation (ASF) under one +//or more contributor license agreements. See the NOTICE file +//distributed with this work for additional information +//regarding copyright ownership. The ASF licenses this file +//to you under the Apache License, Version 2.0 (the +//"License"); you may not use this file except in compliance +//with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +//Unless required by applicable law or agreed to in writing, +//software distributed under the License is distributed on an +//"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +//KIND, either express or implied. See the License for the +//specific language governing permissions and limitations +//under the License. +//*/ +//package org.apache.griffin.measure +// +//import org.apache.griffin.measure.algo._ +//import org.apache.griffin.measure.algo.batch._ +//import org.apache.griffin.measure.algo.streaming._ +//import org.apache.griffin.measure.config.params._ +//import org.apache.griffin.measure.config.params.env._ +//import org.apache.griffin.measure.config.params.user._ +//import org.apache.griffin.measure.config.reader._ +//import org.apache.griffin.measure.config.validator.AllParamValidator +//import org.apache.griffin.measure.log.Loggable +//import org.apache.griffin.measure.persist.PersistThreadPool +// +//import scala.util.{Failure, Success, Try} +// +//object Application extends Loggable { +// +// def main(args: Array[String]): Unit = { +// info(args.toString) +// if (args.length < 2) { +// error("Usage: class [List of String split by comma: raw | local | hdfs(default)]") +// sys.exit(-1) +// } +// +// val envParamFile = args(0) +// val userParamFile = args(1) +// val (envFsType, userFsType) = if (args.length > 2) { +// val fsTypes = args(2).trim.split(",") +// if (fsTypes.length == 1) (fsTypes(0).trim, fsTypes(0).trim) +// else if (fsTypes.length >= 2) (fsTypes(0).trim, fsTypes(1).trim) +// else ("hdfs", "hdfs") +// } else ("hdfs", "hdfs") +// +// info(envParamFile) +// info(userParamFile) +// +// // read param files +// val envParam = readParamFile[EnvParam](envParamFile, envFsType) match { +// case Success(p) => p +// case Failure(ex) => { +// error(ex.getMessage) +// sys.exit(-2) +// } +// } +// val userParam = readParamFile[UserParam](userParamFile, userFsType) match { +// case Success(p) => p +// case Failure(ex) => { +// error(ex.getMessage) +// sys.exit(-2) +// } +// } +// val allParam: AllParam = AllParam(envParam, userParam) +// +// // validate param files +// validateParams(allParam) match { +// case Failure(ex) => { +// error(ex.getMessage) +// sys.exit(-3) +// } +// case _ => { +// info("params validation pass") +// } +// } +// +// // choose algorithm +// val dqType = allParam.userParam.dqType +// val procType = allParam.userParam.procType +// val algo: Algo = (dqType, procType) match { +// case (MeasureType.accuracy(), ProcessType.batch()) => BatchAccuracyAlgo(allParam) +// case (MeasureType.profile(), ProcessType.batch()) => BatchProfileAlgo(allParam) +// case (MeasureType.accuracy(), ProcessType.streaming()) => StreamingAccuracyAlgo(allParam) +//// case (MeasureType.profile(), ProcessType.streaming()) => StreamingProfileAlgo(allParam) +// case _ => { +// error(s"${dqType} with ${procType} is unsupported dq type!") +// sys.exit(-4) +// } +// } +// +// // algorithm run +// algo.run match { +// case Failure(ex) => { +// error(s"app error: ${ex.getMessage}") +// +// procType match { +// case ProcessType.streaming() => { +// // streaming need to attempt more times by spark streaming itself +// throw ex +// } +// case _ => { +// shutdown +// sys.exit(-5) +// } +// } +// } +// case _ => { +// info("app finished and success") +// } +// } +// } +// +// private def readParamFile[T <: Param](file: String, fsType: String)(implicit m : Manifest[T]): Try[T] = { +// val paramReader = ParamReaderFactory.getParamReader(file, fsType) +// paramReader.readConfig[T] +// } +// +// private def validateParams(allParam: AllParam): Try[Boolean] = { +// val allParamValidator = AllParamValidator() +// allParamValidator.validate(allParam) +// } +// +// private def shutdown(): Unit = { +// PersistThreadPool.shutdown +// } +// +//} diff --git a/measure/src/main/scala/org/apache/griffin/measure/algo/batch/BatchAccuracyAlgo.scala b/measure/src/main/scala/org/apache/griffin/measure/algo/batch/BatchAccuracyAlgo.scala index 241f456c0..28366711d 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/algo/batch/BatchAccuracyAlgo.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/algo/batch/BatchAccuracyAlgo.scala @@ -1,190 +1,190 @@ -/* -Licensed to the Apache Software Foundation (ASF) under one -or more contributor license agreements. See the NOTICE file -distributed with this work for additional information -regarding copyright ownership. The ASF licenses this file -to you under the Apache License, Version 2.0 (the -"License"); you may not use this file except in compliance -with the License. You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, -software distributed under the License is distributed on an -"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -KIND, either express or implied. See the License for the -specific language governing permissions and limitations -under the License. -*/ -package org.apache.griffin.measure.algo.batch - -import java.util.Date - -import org.apache.griffin.measure.algo.AccuracyAlgo -import org.apache.griffin.measure.algo.core.AccuracyCore -import org.apache.griffin.measure.config.params.AllParam -import org.apache.griffin.measure.connector._ -import org.apache.griffin.measure.connector.direct.DirectDataConnector -import org.apache.griffin.measure.persist._ -import org.apache.griffin.measure.result._ -import org.apache.griffin.measure.rule._ -import org.apache.griffin.measure.rule.expr._ -import org.apache.spark.rdd.RDD -import org.apache.spark.sql.hive.HiveContext -import org.apache.spark.{SparkConf, SparkContext} - -import scala.util.{Failure, Success, Try} - -// accuracy algorithm for batch mode -case class BatchAccuracyAlgo(allParam: AllParam) extends AccuracyAlgo { - val envParam = allParam.envParam - val userParam = allParam.userParam - - def run(): Try[_] = { - Try { - val metricName = userParam.name - - val sparkParam = envParam.sparkParam - - val conf = new SparkConf().setAppName(metricName) - conf.setAll(sparkParam.config) - val sc = new SparkContext(conf) - sc.setLogLevel(sparkParam.logLevel) - val sqlContext = new HiveContext(sc) - - // start time - val startTime = new Date().getTime() - - // get persists to persist measure result - val persist: Persist = PersistFactory(envParam.persistParams, metricName).getPersists(startTime) - - // get spark application id - val applicationId = sc.applicationId - - // persist start id - persist.start(applicationId) - - // generate rule from rule param, generate rule analyzer - val ruleFactory = RuleFactory(userParam.evaluateRuleParam) - val rule: StatementExpr = ruleFactory.generateRule() - val ruleAnalyzer: RuleAnalyzer = RuleAnalyzer(rule) - - // const expr value map - val constExprValueMap = ExprValueUtil.genExprValueMaps(None, ruleAnalyzer.constCacheExprs, Map[String, Any]()) - val finalConstExprValueMap = ExprValueUtil.updateExprValueMaps(ruleAnalyzer.constFinalCacheExprs, constExprValueMap) - val finalConstMap = finalConstExprValueMap.headOption match { - case Some(m) => m - case _ => Map[String, Any]() - } - - // data connector - val sourceDataConnector: DirectDataConnector = - DataConnectorFactory.getDirectDataConnector(sqlContext, null, userParam.sourceParam, - ruleAnalyzer.sourceRuleExprs, finalConstMap - ) match { - case Success(cntr) => { - if (cntr.available) cntr - else throw new Exception("source data connection error!") - } - case Failure(ex) => throw ex - } - val targetDataConnector: DirectDataConnector = - DataConnectorFactory.getDirectDataConnector(sqlContext, null, userParam.targetParam, - ruleAnalyzer.targetRuleExprs, finalConstMap - ) match { - case Success(cntr) => { - if (cntr.available) cntr - else throw new Exception("target data connection error!") - } - case Failure(ex) => throw ex - } - - // get metadata -// val sourceMetaData: Iterable[(String, String)] = sourceDataConnector.metaData() match { -// case Success(md) => md -// case _ => throw new Exception("source metadata error!") +///* +//Licensed to the Apache Software Foundation (ASF) under one +//or more contributor license agreements. See the NOTICE file +//distributed with this work for additional information +//regarding copyright ownership. The ASF licenses this file +//to you under the Apache License, Version 2.0 (the +//"License"); you may not use this file except in compliance +//with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +//Unless required by applicable law or agreed to in writing, +//software distributed under the License is distributed on an +//"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +//KIND, either express or implied. See the License for the +//specific language governing permissions and limitations +//under the License. +//*/ +//package org.apache.griffin.measure.algo.batch +// +//import java.util.Date +// +//import org.apache.griffin.measure.algo.AccuracyAlgo +//import org.apache.griffin.measure.algo.core.AccuracyCore +//import org.apache.griffin.measure.config.params.AllParam +//import org.apache.griffin.measure.connector._ +//import org.apache.griffin.measure.connector.direct.DirectDataConnector +//import org.apache.griffin.measure.persist._ +//import org.apache.griffin.measure.result._ +//import org.apache.griffin.measure.rule._ +//import org.apache.griffin.measure.rule.expr._ +//import org.apache.spark.rdd.RDD +//import org.apache.spark.sql.hive.HiveContext +//import org.apache.spark.{SparkConf, SparkContext} +// +//import scala.util.{Failure, Success, Try} +// +//// accuracy algorithm for batch mode +//case class BatchAccuracyAlgo(allParam: AllParam) extends AccuracyAlgo { +// val envParam = allParam.envParam +// val userParam = allParam.userParam +// +// def run(): Try[_] = { +// Try { +// val metricName = userParam.name +// +// val sparkParam = envParam.sparkParam +// +// val conf = new SparkConf().setAppName(metricName) +// conf.setAll(sparkParam.config) +// val sc = new SparkContext(conf) +// sc.setLogLevel(sparkParam.logLevel) +// val sqlContext = new HiveContext(sc) +// +// // start time +// val startTime = new Date().getTime() +// +// // get persists to persist measure result +// val persist: Persist = PersistFactory(envParam.persistParams, metricName).getPersists(startTime) +// +// // get spark application id +// val applicationId = sc.applicationId +// +// // persist start id +// persist.start(applicationId) +// +// // generate rule from rule param, generate rule analyzer +// val ruleFactory = RuleFactory(userParam.evaluateRuleParam) +// val rule: StatementExpr = ruleFactory.generateRule() +// val ruleAnalyzer: RuleAnalyzer = RuleAnalyzer(rule) +// +// // const expr value map +// val constExprValueMap = ExprValueUtil.genExprValueMaps(None, ruleAnalyzer.constCacheExprs, Map[String, Any]()) +// val finalConstExprValueMap = ExprValueUtil.updateExprValueMaps(ruleAnalyzer.constFinalCacheExprs, constExprValueMap) +// val finalConstMap = finalConstExprValueMap.headOption match { +// case Some(m) => m +// case _ => Map[String, Any]() // } -// val targetMetaData: Iterable[(String, String)] = targetDataConnector.metaData() match { -// case Success(md) => md -// case _ => throw new Exception("target metadata error!") +// +// // data connector +// val sourceDataConnector: DirectDataConnector = +// DataConnectorFactory.getDirectDataConnector(sqlContext, null, userParam.sourceParam, +// ruleAnalyzer.sourceRuleExprs, finalConstMap +// ) match { +// case Success(cntr) => { +// if (cntr.available) cntr +// else throw new Exception("source data connection error!") +// } +// case Failure(ex) => throw ex +// } +// val targetDataConnector: DirectDataConnector = +// DataConnectorFactory.getDirectDataConnector(sqlContext, null, userParam.targetParam, +// ruleAnalyzer.targetRuleExprs, finalConstMap +// ) match { +// case Success(cntr) => { +// if (cntr.available) cntr +// else throw new Exception("target data connection error!") +// } +// case Failure(ex) => throw ex +// } +// +// // get metadata +//// val sourceMetaData: Iterable[(String, String)] = sourceDataConnector.metaData() match { +//// case Success(md) => md +//// case _ => throw new Exception("source metadata error!") +//// } +//// val targetMetaData: Iterable[(String, String)] = targetDataConnector.metaData() match { +//// case Success(md) => md +//// case _ => throw new Exception("target metadata error!") +//// } +// +// // get data +// val sourceData: RDD[(Product, (Map[String, Any], Map[String, Any]))] = sourceDataConnector.data() match { +// case Success(dt) => dt +// case Failure(ex) => throw ex // } - - // get data - val sourceData: RDD[(Product, (Map[String, Any], Map[String, Any]))] = sourceDataConnector.data() match { - case Success(dt) => dt - case Failure(ex) => throw ex - } - val targetData: RDD[(Product, (Map[String, Any], Map[String, Any]))] = targetDataConnector.data() match { - case Success(dt) => dt - case Failure(ex) => throw ex - } - - // accuracy algorithm - val (accuResult, missingRdd, matchedRdd) = accuracy(sourceData, targetData, ruleAnalyzer) - - // end time - val endTime = new Date().getTime - persist.log(endTime, s"calculation using time: ${endTime - startTime} ms") - - // persist result - persist.result(endTime, accuResult) - val missingRecords = missingRdd.map(record2String(_, ruleAnalyzer.sourceRuleExprs.persistExprs, ruleAnalyzer.targetRuleExprs.persistExprs)) -// persist.missRecords(missingRecords) - persist.records(missingRecords, PersistType.MISS) - - // persist end time - val persistEndTime = new Date().getTime - persist.log(persistEndTime, s"persist using time: ${persistEndTime - endTime} ms") - - // finish - persist.finish() - - // context stop - sc.stop - - } - } - - def wrapInitData(data: Map[String, Any]): (Map[String, Any], Map[String, Any]) = { - (data, Map[String, Any]()) - } - - // calculate accuracy between source data and target data - def accuracy(sourceData: RDD[(Product, (Map[String, Any], Map[String, Any]))], - targetData: RDD[(Product, (Map[String, Any], Map[String, Any]))], - ruleAnalyzer: RuleAnalyzer) = { - // 1. cogroup - val allKvs = sourceData.cogroup(targetData) - - // 2. accuracy calculation - val (accuResult, missingRdd, matchedRdd) = AccuracyCore.accuracy(allKvs, ruleAnalyzer) - - (accuResult, missingRdd, matchedRdd) - } - - // convert data into a string - def record2String(rec: (Product, (Map[String, Any], Map[String, Any])), sourcePersist: Iterable[Expr], targetPersist: Iterable[Expr]): String = { - val (key, (data, info)) = rec - val persistData = getPersistMap(data, sourcePersist) - val persistInfo = info.mapValues { value => - value match { - case vd: Map[String, Any] => getPersistMap(vd, targetPersist) - case v => v - } - }.map(identity) - s"${persistData} [${persistInfo}]" - } - - // get the expr value map of the persist expressions - private def getPersistMap(data: Map[String, Any], persist: Iterable[Expr]): Map[String, Any] = { - val persistMap = persist.map(e => (e._id, e.desc)).toMap - data.flatMap { pair => - val (k, v) = pair - persistMap.get(k) match { - case Some(d) => Some((d -> v)) - case _ => None - } - } - } - -} +// val targetData: RDD[(Product, (Map[String, Any], Map[String, Any]))] = targetDataConnector.data() match { +// case Success(dt) => dt +// case Failure(ex) => throw ex +// } +// +// // accuracy algorithm +// val (accuResult, missingRdd, matchedRdd) = accuracy(sourceData, targetData, ruleAnalyzer) +// +// // end time +// val endTime = new Date().getTime +// persist.log(endTime, s"calculation using time: ${endTime - startTime} ms") +// +// // persist result +// persist.result(endTime, accuResult) +// val missingRecords = missingRdd.map(record2String(_, ruleAnalyzer.sourceRuleExprs.persistExprs, ruleAnalyzer.targetRuleExprs.persistExprs)) +//// persist.missRecords(missingRecords) +// persist.records(missingRecords, PersistType.MISS) +// +// // persist end time +// val persistEndTime = new Date().getTime +// persist.log(persistEndTime, s"persist using time: ${persistEndTime - endTime} ms") +// +// // finish +// persist.finish() +// +// // context stop +// sc.stop +// +// } +// } +// +// def wrapInitData(data: Map[String, Any]): (Map[String, Any], Map[String, Any]) = { +// (data, Map[String, Any]()) +// } +// +// // calculate accuracy between source data and target data +// def accuracy(sourceData: RDD[(Product, (Map[String, Any], Map[String, Any]))], +// targetData: RDD[(Product, (Map[String, Any], Map[String, Any]))], +// ruleAnalyzer: RuleAnalyzer) = { +// // 1. cogroup +// val allKvs = sourceData.cogroup(targetData) +// +// // 2. accuracy calculation +// val (accuResult, missingRdd, matchedRdd) = AccuracyCore.accuracy(allKvs, ruleAnalyzer) +// +// (accuResult, missingRdd, matchedRdd) +// } +// +// // convert data into a string +// def record2String(rec: (Product, (Map[String, Any], Map[String, Any])), sourcePersist: Iterable[Expr], targetPersist: Iterable[Expr]): String = { +// val (key, (data, info)) = rec +// val persistData = getPersistMap(data, sourcePersist) +// val persistInfo = info.mapValues { value => +// value match { +// case vd: Map[String, Any] => getPersistMap(vd, targetPersist) +// case v => v +// } +// }.map(identity) +// s"${persistData} [${persistInfo}]" +// } +// +// // get the expr value map of the persist expressions +// private def getPersistMap(data: Map[String, Any], persist: Iterable[Expr]): Map[String, Any] = { +// val persistMap = persist.map(e => (e._id, e.desc)).toMap +// data.flatMap { pair => +// val (k, v) = pair +// persistMap.get(k) match { +// case Some(d) => Some((d -> v)) +// case _ => None +// } +// } +// } +// +//} diff --git a/measure/src/main/scala/org/apache/griffin/measure/algo/batch/BatchProfileAlgo.scala b/measure/src/main/scala/org/apache/griffin/measure/algo/batch/BatchProfileAlgo.scala index 163a0b709..c8c1171ef 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/algo/batch/BatchProfileAlgo.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/algo/batch/BatchProfileAlgo.scala @@ -1,162 +1,162 @@ -/* -Licensed to the Apache Software Foundation (ASF) under one -or more contributor license agreements. See the NOTICE file -distributed with this work for additional information -regarding copyright ownership. The ASF licenses this file -to you under the Apache License, Version 2.0 (the -"License"); you may not use this file except in compliance -with the License. You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, -software distributed under the License is distributed on an -"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -KIND, either express or implied. See the License for the -specific language governing permissions and limitations -under the License. -*/ -package org.apache.griffin.measure.algo.batch - -import java.util.Date - -import org.apache.griffin.measure.algo.ProfileAlgo -import org.apache.griffin.measure.algo.core.ProfileCore -import org.apache.griffin.measure.config.params._ -import org.apache.griffin.measure.connector._ -import org.apache.griffin.measure.connector.direct.DirectDataConnector -import org.apache.griffin.measure.persist.{Persist, PersistFactory, PersistType} -import org.apache.griffin.measure.result._ -import org.apache.griffin.measure.rule.expr._ -import org.apache.griffin.measure.rule.{ExprValueUtil, RuleAnalyzer, RuleFactory} -import org.apache.spark.rdd.RDD -import org.apache.spark.sql.hive.HiveContext -import org.apache.spark.{SparkConf, SparkContext} - -import scala.util.{Failure, Success, Try} - -// profile algorithm for batch mode -case class BatchProfileAlgo(allParam: AllParam) extends ProfileAlgo { - val envParam = allParam.envParam - val userParam = allParam.userParam - - def run(): Try[_] = { - Try { - val metricName = userParam.name - - val sparkParam = envParam.sparkParam - - val conf = new SparkConf().setAppName(metricName) - conf.setAll(sparkParam.config) - val sc = new SparkContext(conf) - sc.setLogLevel(sparkParam.logLevel) - val sqlContext = new HiveContext(sc) - - // start time - val startTime = new Date().getTime() - - // get persists to persist measure result - val persist: Persist = PersistFactory(envParam.persistParams, metricName).getPersists(startTime) - - // get spark application id - val applicationId = sc.applicationId - - // persist start id - persist.start(applicationId) - - // generate rule from rule param, generate rule analyzer - val ruleFactory = RuleFactory(userParam.evaluateRuleParam) - val rule: StatementExpr = ruleFactory.generateRule() - val ruleAnalyzer: RuleAnalyzer = RuleAnalyzer(rule) - - // const expr value map - val constExprValueMap = ExprValueUtil.genExprValueMaps(None, ruleAnalyzer.constCacheExprs, Map[String, Any]()) - val finalConstExprValueMap = ExprValueUtil.updateExprValueMaps(ruleAnalyzer.constFinalCacheExprs, constExprValueMap) - val finalConstMap = finalConstExprValueMap.headOption match { - case Some(m) => m - case _ => Map[String, Any]() - } - - // data connector - val sourceDataConnector: DirectDataConnector = - DataConnectorFactory.getDirectDataConnector(sqlContext, null, userParam.sourceParam, - ruleAnalyzer.sourceRuleExprs, finalConstMap - ) match { - case Success(cntr) => { - if (cntr.available) cntr - else throw new Exception("source data connection error!") - } - case Failure(ex) => throw ex - } - - // get metadata - // val sourceMetaData: Iterable[(String, String)] = sourceDataConnector.metaData() match { - // case Success(md) => md - // case _ => throw new Exception("source metadata error!") - // } - - // get data - val sourceData: RDD[(Product, (Map[String, Any], Map[String, Any]))] = sourceDataConnector.data() match { - case Success(dt) => dt - case Failure(ex) => throw ex - } - - // profile algorithm - val (profileResult, missingRdd, matchedRdd) = profile(sourceData, ruleAnalyzer) - - // end time - val endTime = new Date().getTime - persist.log(endTime, s"calculation using time: ${endTime - startTime} ms") - - // persist result - persist.result(endTime, profileResult) - val matchedRecords = matchedRdd.map(record2String(_, ruleAnalyzer.sourceRuleExprs.persistExprs)) -// persist.matchRecords(matchedRecords) - persist.records(matchedRecords, PersistType.MATCH) - - // persist end time - val persistEndTime = new Date().getTime - persist.log(persistEndTime, s"persist using time: ${persistEndTime - endTime} ms") - - // finish - persist.finish() - - // context stop - sc.stop - } - } - - def wrapInitData(data: Map[String, Any]): (Map[String, Any], Map[String, Any]) = { - (data, Map[String, Any]()) - } - - // calculate profile from source data - def profile(sourceData: RDD[(Product, (Map[String, Any], Map[String, Any]))], ruleAnalyzer: RuleAnalyzer - ) = { - // 1. profile calculation - val (profileResult, missingRdd, matchedRdd) = ProfileCore.profile(sourceData, ruleAnalyzer) - - (profileResult, missingRdd, matchedRdd) - } - - // convert data into a string - def record2String(rec: (Product, (Map[String, Any], Map[String, Any])), sourcePersist: Iterable[Expr]): String = { - val (key, (data, info)) = rec - val persistData = getPersistMap(data, sourcePersist) - val persistInfo = info - if (persistInfo.size > 0) s"${persistData} [${persistInfo}]" else s"${persistData}" - } - - // get the expr value map of the persist expressions - private def getPersistMap(data: Map[String, Any], persist: Iterable[Expr]): Map[String, Any] = { - val persistMap = persist.map(e => (e._id, e.desc)).toMap - data.flatMap { pair => - val (k, v) = pair - persistMap.get(k) match { - case Some(d) => Some((d -> v)) - case _ => None - } - } - } - -} +///* +//Licensed to the Apache Software Foundation (ASF) under one +//or more contributor license agreements. See the NOTICE file +//distributed with this work for additional information +//regarding copyright ownership. The ASF licenses this file +//to you under the Apache License, Version 2.0 (the +//"License"); you may not use this file except in compliance +//with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +//Unless required by applicable law or agreed to in writing, +//software distributed under the License is distributed on an +//"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +//KIND, either express or implied. See the License for the +//specific language governing permissions and limitations +//under the License. +//*/ +//package org.apache.griffin.measure.algo.batch +// +//import java.util.Date +// +//import org.apache.griffin.measure.algo.ProfileAlgo +//import org.apache.griffin.measure.algo.core.ProfileCore +//import org.apache.griffin.measure.config.params._ +//import org.apache.griffin.measure.connector._ +//import org.apache.griffin.measure.connector.direct.DirectDataConnector +//import org.apache.griffin.measure.persist.{Persist, PersistFactory, PersistType} +//import org.apache.griffin.measure.result._ +//import org.apache.griffin.measure.rule.expr._ +//import org.apache.griffin.measure.rule.{ExprValueUtil, RuleAnalyzer, RuleFactory} +//import org.apache.spark.rdd.RDD +//import org.apache.spark.sql.hive.HiveContext +//import org.apache.spark.{SparkConf, SparkContext} +// +//import scala.util.{Failure, Success, Try} +// +//// profile algorithm for batch mode +//case class BatchProfileAlgo(allParam: AllParam) extends ProfileAlgo { +// val envParam = allParam.envParam +// val userParam = allParam.userParam +// +// def run(): Try[_] = { +// Try { +// val metricName = userParam.name +// +// val sparkParam = envParam.sparkParam +// +// val conf = new SparkConf().setAppName(metricName) +// conf.setAll(sparkParam.config) +// val sc = new SparkContext(conf) +// sc.setLogLevel(sparkParam.logLevel) +// val sqlContext = new HiveContext(sc) +// +// // start time +// val startTime = new Date().getTime() +// +// // get persists to persist measure result +// val persist: Persist = PersistFactory(envParam.persistParams, metricName).getPersists(startTime) +// +// // get spark application id +// val applicationId = sc.applicationId +// +// // persist start id +// persist.start(applicationId) +// +// // generate rule from rule param, generate rule analyzer +// val ruleFactory = RuleFactory(userParam.evaluateRuleParam) +// val rule: StatementExpr = ruleFactory.generateRule() +// val ruleAnalyzer: RuleAnalyzer = RuleAnalyzer(rule) +// +// // const expr value map +// val constExprValueMap = ExprValueUtil.genExprValueMaps(None, ruleAnalyzer.constCacheExprs, Map[String, Any]()) +// val finalConstExprValueMap = ExprValueUtil.updateExprValueMaps(ruleAnalyzer.constFinalCacheExprs, constExprValueMap) +// val finalConstMap = finalConstExprValueMap.headOption match { +// case Some(m) => m +// case _ => Map[String, Any]() +// } +// +// // data connector +// val sourceDataConnector: DirectDataConnector = +// DataConnectorFactory.getDirectDataConnector(sqlContext, null, userParam.sourceParam, +// ruleAnalyzer.sourceRuleExprs, finalConstMap +// ) match { +// case Success(cntr) => { +// if (cntr.available) cntr +// else throw new Exception("source data connection error!") +// } +// case Failure(ex) => throw ex +// } +// +// // get metadata +// // val sourceMetaData: Iterable[(String, String)] = sourceDataConnector.metaData() match { +// // case Success(md) => md +// // case _ => throw new Exception("source metadata error!") +// // } +// +// // get data +// val sourceData: RDD[(Product, (Map[String, Any], Map[String, Any]))] = sourceDataConnector.data() match { +// case Success(dt) => dt +// case Failure(ex) => throw ex +// } +// +// // profile algorithm +// val (profileResult, missingRdd, matchedRdd) = profile(sourceData, ruleAnalyzer) +// +// // end time +// val endTime = new Date().getTime +// persist.log(endTime, s"calculation using time: ${endTime - startTime} ms") +// +// // persist result +// persist.result(endTime, profileResult) +// val matchedRecords = matchedRdd.map(record2String(_, ruleAnalyzer.sourceRuleExprs.persistExprs)) +//// persist.matchRecords(matchedRecords) +// persist.records(matchedRecords, PersistType.MATCH) +// +// // persist end time +// val persistEndTime = new Date().getTime +// persist.log(persistEndTime, s"persist using time: ${persistEndTime - endTime} ms") +// +// // finish +// persist.finish() +// +// // context stop +// sc.stop +// } +// } +// +// def wrapInitData(data: Map[String, Any]): (Map[String, Any], Map[String, Any]) = { +// (data, Map[String, Any]()) +// } +// +// // calculate profile from source data +// def profile(sourceData: RDD[(Product, (Map[String, Any], Map[String, Any]))], ruleAnalyzer: RuleAnalyzer +// ) = { +// // 1. profile calculation +// val (profileResult, missingRdd, matchedRdd) = ProfileCore.profile(sourceData, ruleAnalyzer) +// +// (profileResult, missingRdd, matchedRdd) +// } +// +// // convert data into a string +// def record2String(rec: (Product, (Map[String, Any], Map[String, Any])), sourcePersist: Iterable[Expr]): String = { +// val (key, (data, info)) = rec +// val persistData = getPersistMap(data, sourcePersist) +// val persistInfo = info +// if (persistInfo.size > 0) s"${persistData} [${persistInfo}]" else s"${persistData}" +// } +// +// // get the expr value map of the persist expressions +// private def getPersistMap(data: Map[String, Any], persist: Iterable[Expr]): Map[String, Any] = { +// val persistMap = persist.map(e => (e._id, e.desc)).toMap +// data.flatMap { pair => +// val (k, v) = pair +// persistMap.get(k) match { +// case Some(d) => Some((d -> v)) +// case _ => None +// } +// } +// } +// +//} diff --git a/measure/src/main/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgo.scala b/measure/src/main/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgo.scala index 41142755a..810356106 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgo.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgo.scala @@ -1,369 +1,369 @@ -/* -Licensed to the Apache Software Foundation (ASF) under one -or more contributor license agreements. See the NOTICE file -distributed with this work for additional information -regarding copyright ownership. The ASF licenses this file -to you under the Apache License, Version 2.0 (the -"License"); you may not use this file except in compliance -with the License. You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, -software distributed under the License is distributed on an -"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -KIND, either express or implied. See the License for the -specific language governing permissions and limitations -under the License. -*/ -package org.apache.griffin.measure.algo.streaming - -import java.util.Date -import java.util.concurrent.{Executors, ThreadPoolExecutor, TimeUnit} - -import org.apache.griffin.measure.algo.AccuracyAlgo -import org.apache.griffin.measure.algo.core.AccuracyCore -import org.apache.griffin.measure.cache.info.{InfoCacheInstance, TimeInfoCache} -import org.apache.griffin.measure.cache.result.CacheResultProcesser -import org.apache.griffin.measure.config.params.AllParam -import org.apache.griffin.measure.connector._ -import org.apache.griffin.measure.connector.direct.DirectDataConnector -import org.apache.griffin.measure.persist.{Persist, PersistFactory, PersistType} -import org.apache.griffin.measure.result.{AccuracyResult, MismatchInfo, TimeStampInfo} -import org.apache.griffin.measure.rule.{ExprValueUtil, RuleAnalyzer, RuleFactory} -import org.apache.griffin.measure.rule.expr._ -import org.apache.griffin.measure.utils.TimeUtil -import org.apache.spark.rdd.RDD -import org.apache.spark.sql.SQLContext -import org.apache.spark.sql.hive.HiveContext -import org.apache.spark.streaming.{Milliseconds, Seconds, StreamingContext} -import org.apache.spark.{SparkConf, SparkContext} - -import scala.util.{Failure, Success, Try} - - -case class StreamingAccuracyAlgo(allParam: AllParam) extends AccuracyAlgo { - val envParam = allParam.envParam - val userParam = allParam.userParam - - def run(): Try[_] = { - Try { - val metricName = userParam.name - - val sparkParam = envParam.sparkParam - - val conf = new SparkConf().setAppName(metricName) - conf.setAll(sparkParam.config) - val sc = new SparkContext(conf) - sc.setLogLevel(sparkParam.logLevel) - val sqlContext = new HiveContext(sc) -// val sqlContext = new SQLContext(sc) - -// val batchInterval = TimeUtil.milliseconds(sparkParam.batchInterval) match { -// case Some(interval) => Milliseconds(interval) -// case _ => throw new Exception("invalid batch interval") +///* +//Licensed to the Apache Software Foundation (ASF) under one +//or more contributor license agreements. See the NOTICE file +//distributed with this work for additional information +//regarding copyright ownership. The ASF licenses this file +//to you under the Apache License, Version 2.0 (the +//"License"); you may not use this file except in compliance +//with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +//Unless required by applicable law or agreed to in writing, +//software distributed under the License is distributed on an +//"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +//KIND, either express or implied. See the License for the +//specific language governing permissions and limitations +//under the License. +//*/ +//package org.apache.griffin.measure.algo.streaming +// +//import java.util.Date +//import java.util.concurrent.{Executors, ThreadPoolExecutor, TimeUnit} +// +//import org.apache.griffin.measure.algo.AccuracyAlgo +//import org.apache.griffin.measure.algo.core.AccuracyCore +//import org.apache.griffin.measure.cache.info.{InfoCacheInstance, TimeInfoCache} +//import org.apache.griffin.measure.cache.result.CacheResultProcesser +//import org.apache.griffin.measure.config.params.AllParam +//import org.apache.griffin.measure.connector._ +//import org.apache.griffin.measure.connector.direct.DirectDataConnector +//import org.apache.griffin.measure.persist.{Persist, PersistFactory, PersistType} +//import org.apache.griffin.measure.result.{AccuracyResult, MismatchInfo, TimeStampInfo} +//import org.apache.griffin.measure.rule.{ExprValueUtil, RuleAnalyzer, RuleFactory} +//import org.apache.griffin.measure.rule.expr._ +//import org.apache.griffin.measure.utils.TimeUtil +//import org.apache.spark.rdd.RDD +//import org.apache.spark.sql.SQLContext +//import org.apache.spark.sql.hive.HiveContext +//import org.apache.spark.streaming.{Milliseconds, Seconds, StreamingContext} +//import org.apache.spark.{SparkConf, SparkContext} +// +//import scala.util.{Failure, Success, Try} +// +// +//case class StreamingAccuracyAlgo(allParam: AllParam) extends AccuracyAlgo { +// val envParam = allParam.envParam +// val userParam = allParam.userParam +// +// def run(): Try[_] = { +// Try { +// val metricName = userParam.name +// +// val sparkParam = envParam.sparkParam +// +// val conf = new SparkConf().setAppName(metricName) +// conf.setAll(sparkParam.config) +// val sc = new SparkContext(conf) +// sc.setLogLevel(sparkParam.logLevel) +// val sqlContext = new HiveContext(sc) +//// val sqlContext = new SQLContext(sc) +// +//// val batchInterval = TimeUtil.milliseconds(sparkParam.batchInterval) match { +//// case Some(interval) => Milliseconds(interval) +//// case _ => throw new Exception("invalid batch interval") +//// } +//// val ssc = new StreamingContext(sc, batchInterval) +//// ssc.checkpoint(sparkParam.cpDir) +// +// def createStreamingContext(): StreamingContext = { +// val batchInterval = TimeUtil.milliseconds(sparkParam.batchInterval) match { +// case Some(interval) => Milliseconds(interval) +// case _ => throw new Exception("invalid batch interval") +// } +// val ssc = new StreamingContext(sc, batchInterval) +// ssc.checkpoint(sparkParam.cpDir) +// ssc // } -// val ssc = new StreamingContext(sc, batchInterval) -// ssc.checkpoint(sparkParam.cpDir) - - def createStreamingContext(): StreamingContext = { - val batchInterval = TimeUtil.milliseconds(sparkParam.batchInterval) match { - case Some(interval) => Milliseconds(interval) - case _ => throw new Exception("invalid batch interval") - } - val ssc = new StreamingContext(sc, batchInterval) - ssc.checkpoint(sparkParam.cpDir) - ssc - } - val ssc = StreamingContext.getOrCreate(sparkParam.cpDir, createStreamingContext) - - // init info cache instance - InfoCacheInstance.initInstance(envParam.infoCacheParams, metricName) - InfoCacheInstance.init - - // start time - val startTime = new Date().getTime() - - val persistFactory = PersistFactory(envParam.persistParams, metricName) - - // get persists to persist measure result - val appPersist: Persist = persistFactory.getPersists(startTime) - - // get spark application id - val applicationId = sc.applicationId - - // persist start id - appPersist.start(applicationId) - - // generate rule from rule param, generate rule analyzer - val ruleFactory = RuleFactory(userParam.evaluateRuleParam) - val rule: StatementExpr = ruleFactory.generateRule() - val ruleAnalyzer: RuleAnalyzer = RuleAnalyzer(rule) - - // const expr value map - val constExprValueMap = ExprValueUtil.genExprValueMaps(None, ruleAnalyzer.constCacheExprs, Map[String, Any]()) - val finalConstExprValueMap = ExprValueUtil.updateExprValueMaps(ruleAnalyzer.constFinalCacheExprs, constExprValueMap) - val finalConstMap = finalConstExprValueMap.headOption match { - case Some(m) => m - case _ => Map[String, Any]() - } - - // data connector - val sourceDataConnector: DirectDataConnector = - DataConnectorFactory.getDirectDataConnector(sqlContext, ssc, userParam.sourceParam, - ruleAnalyzer.sourceRuleExprs, finalConstMap - ) match { - case Success(cntr) => { - if (cntr.available) cntr - else throw new Exception("source data connection error!") - } - case Failure(ex) => throw ex - } - val targetDataConnector: DirectDataConnector = - DataConnectorFactory.getDirectDataConnector(sqlContext, ssc, userParam.targetParam, - ruleAnalyzer.targetRuleExprs, finalConstMap - ) match { - case Success(cntr) => { - if (cntr.available) cntr - else throw new Exception("target data connection error!") - } - case Failure(ex) => throw ex - } - - val cacheResultProcesser = CacheResultProcesser() - - // init data stream - sourceDataConnector.init() - targetDataConnector.init() - - val streamingAccuracyProcess = StreamingAccuracyProcess( - sourceDataConnector, targetDataConnector, - ruleAnalyzer, cacheResultProcesser, persistFactory, appPersist) - - // process thread -// case class Process() extends Runnable { -// val lock = InfoCacheInstance.genLock("process") -// def run(): Unit = { -// val updateTime = new Date().getTime -// val locked = lock.lock(5, TimeUnit.SECONDS) -// if (locked) { -// try { -// val st = new Date().getTime -// -// TimeInfoCache.startTimeInfoCache -// -// // get data -// val sourceData = sourceDataConnector.data match { -// case Success(dt) => dt -// case Failure(ex) => throw ex -// } -// val targetData = targetDataConnector.data match { -// case Success(dt) => dt -// case Failure(ex) => throw ex -// } -// -// sourceData.cache -// targetData.cache -// -// println(s"sourceData.count: ${sourceData.count}") -// println(s"targetData.count: ${targetData.count}") -// -// // accuracy algorithm -// val (accuResult, missingRdd, matchedRdd) = accuracy(sourceData, targetData, ruleAnalyzer) -// println(s"accuResult: ${accuResult}") -// -// val ct = new Date().getTime -// appPersist.log(ct, s"calculation using time: ${ct - st} ms") -// -// sourceData.unpersist() -// targetData.unpersist() -// -// // result of every group -// val matchedGroups = reorgByTimeGroup(matchedRdd) -// val matchedGroupCount = matchedGroups.count -// println(s"===== matchedGroupCount: ${matchedGroupCount} =====") -// -// // get missing results -// val missingGroups = reorgByTimeGroup(missingRdd) -// val missingGroupCount = missingGroups.count -// println(s"===== missingGroupCount: ${missingGroupCount} =====") -// -// val groups = matchedGroups.cogroup(missingGroups) -// val groupCount = groups.count -// println(s"===== groupCount: ${groupCount} =====") -// -// val updateResults = groups.flatMap { group => -// val (t, (matchData, missData)) = group -// -// val matchSize = matchData.size -// val missSize = missData.size -// val res = AccuracyResult(missSize, matchSize + missSize) -// -// val updatedCacheResultOpt = cacheResultProcesser.genUpdateCacheResult(t, updateTime, res) -// -// updatedCacheResultOpt.flatMap { updatedCacheResult => -// Some((updatedCacheResult, (t, missData))) -// } -// } -// -// updateResults.cache -// -// val updateResultsPart = updateResults.map(_._1) -// val updateDataPart = updateResults.map(_._2) -// -// val updateResultsArray = updateResultsPart.collect() -// -// // update results cache (in driver) -// // collect action is traversable once action, it will make rdd updateResults empty -// updateResultsArray.foreach { updateResult => -// println(s"update result: ${updateResult}") -// cacheResultProcesser.update(updateResult) -// // persist result -// val persist: Persist = persistFactory.getPersists(updateResult.timeGroup) -// persist.result(updateTime, updateResult.result) -// } -// -// // record missing data and update old data (in executor) -// updateDataPart.foreach { grp => -// val (t, datas) = grp -// val persist: Persist = persistFactory.getPersists(t) -// // persist missing data -// val missStrings = datas.map { row => -// val (_, (value, info)) = row -// s"${value} [${info.getOrElse(MismatchInfo.key, "unknown")}]" -// } -// persist.records(missStrings, PersistType.MISS) -// // data connector update old data -// val dumpDatas = datas.map { r => -// val (_, (v, i)) = r -// v ++ i -// } -// -// println(t) -// dumpDatas.foreach(println) -// -// sourceDataConnector.updateOldData(t, dumpDatas) -// targetDataConnector.updateOldData(t, dumpDatas) // not correct -// } -// -// updateResults.unpersist() -// -// // dump missing rdd (this part not need for future version, only for current df cache data version) -// val dumpRdd: RDD[Map[String, Any]] = missingRdd.map { r => -// val (_, (v, i)) = r -// v ++ i -// } -// sourceDataConnector.updateAllOldData(dumpRdd) -// targetDataConnector.updateAllOldData(dumpRdd) // not correct -// -// TimeInfoCache.endTimeInfoCache -// -// val et = new Date().getTime -// appPersist.log(et, s"persist using time: ${et - ct} ms") -// -// } catch { -// case e: Throwable => error(s"process error: ${e.getMessage}") -// } finally { -// lock.unlock() -// } -// } +// val ssc = StreamingContext.getOrCreate(sparkParam.cpDir, createStreamingContext) +// +// // init info cache instance +// InfoCacheInstance.initInstance(envParam.infoCacheParams, metricName) +// InfoCacheInstance.init +// +// // start time +// val startTime = new Date().getTime() +// +// val persistFactory = PersistFactory(envParam.persistParams, metricName) +// +// // get persists to persist measure result +// val appPersist: Persist = persistFactory.getPersists(startTime) +// +// // get spark application id +// val applicationId = sc.applicationId +// +// // persist start id +// appPersist.start(applicationId) +// +// // generate rule from rule param, generate rule analyzer +// val ruleFactory = RuleFactory(userParam.evaluateRuleParam) +// val rule: StatementExpr = ruleFactory.generateRule() +// val ruleAnalyzer: RuleAnalyzer = RuleAnalyzer(rule) +// +// // const expr value map +// val constExprValueMap = ExprValueUtil.genExprValueMaps(None, ruleAnalyzer.constCacheExprs, Map[String, Any]()) +// val finalConstExprValueMap = ExprValueUtil.updateExprValueMaps(ruleAnalyzer.constFinalCacheExprs, constExprValueMap) +// val finalConstMap = finalConstExprValueMap.headOption match { +// case Some(m) => m +// case _ => Map[String, Any]() +// } +// +// // data connector +// val sourceDataConnector: DirectDataConnector = +// DataConnectorFactory.getDirectDataConnector(sqlContext, ssc, userParam.sourceParam, +// ruleAnalyzer.sourceRuleExprs, finalConstMap +// ) match { +// case Success(cntr) => { +// if (cntr.available) cntr +// else throw new Exception("source data connection error!") // } +// case Failure(ex) => throw ex // } - - val processInterval = TimeUtil.milliseconds(sparkParam.processInterval) match { - case Some(interval) => interval - case _ => throw new Exception("invalid batch interval") - } - val process = TimingProcess(processInterval, streamingAccuracyProcess) - - // clean thread -// case class Clean() extends Runnable { -// val lock = InfoCacheInstance.genLock("clean") -// def run(): Unit = { -// val locked = lock.lock(5, TimeUnit.SECONDS) -// if (locked) { -// try { -// sourceDataConnector.cleanData -// targetDataConnector.cleanData -// } finally { -// lock.unlock() +// val targetDataConnector: DirectDataConnector = +// DataConnectorFactory.getDirectDataConnector(sqlContext, ssc, userParam.targetParam, +// ruleAnalyzer.targetRuleExprs, finalConstMap +// ) match { +// case Success(cntr) => { +// if (cntr.available) cntr +// else throw new Exception("target data connection error!") // } +// case Failure(ex) => throw ex // } -// } -// } -// val cleanInterval = TimeUtil.milliseconds(cleanerParam.cleanInterval) match { -// case Some(interval) => interval -// case _ => throw new Exception("invalid batch interval") -// } -// val clean = TimingProcess(cleanInterval, Clean()) - - process.startup() -// clean.startup() - - ssc.start() - ssc.awaitTermination() - ssc.stop(stopSparkContext=true, stopGracefully=true) - - // context stop - sc.stop - - InfoCacheInstance.close - - appPersist.finish() - - process.shutdown() -// clean.shutdown() - } - } - - // calculate accuracy between source data and target data -// def accuracy(sourceData: RDD[(Product, (Map[String, Any], Map[String, Any]))], -// targetData: RDD[(Product, (Map[String, Any], Map[String, Any]))], -// ruleAnalyzer: RuleAnalyzer) = { -// // 1. cogroup -// val allKvs = sourceData.cogroup(targetData) -// -// // 2. accuracy calculation -// val (accuResult, missingRdd, matchedRdd) = AccuracyCore.accuracy(allKvs, ruleAnalyzer) -// -// (accuResult, missingRdd, matchedRdd) -// } - -// // convert data into a string -// def record2String(rec: (Product, (Map[String, Any], Map[String, Any])), sourcePersist: Iterable[Expr], targetPersist: Iterable[Expr]): String = { -// val (key, (data, info)) = rec -// val persistData = getPersistMap(data, sourcePersist) -// val persistInfo = info.mapValues { value => -// value match { -// case vd: Map[String, Any] => getPersistMap(vd, targetPersist) -// case v => v -// } -// }.map(identity) -// s"${persistData} [${persistInfo}]" -// } // -// // get the expr value map of the persist expressions -// private def getPersistMap(data: Map[String, Any], persist: Iterable[Expr]): Map[String, Any] = { -// val persistMap = persist.map(e => (e._id, e.desc)).toMap -// data.flatMap { pair => -// val (k, v) = pair -// persistMap.get(k) match { -// case Some(d) => Some((d -> v)) -// case _ => None -// } -// } -// } - -// def reorgByTimeGroup(rdd: RDD[(Product, (Map[String, Any], Map[String, Any]))] -// ): RDD[(Long, (Product, (Map[String, Any], Map[String, Any])))] = { -// rdd.flatMap { row => -// val (key, (value, info)) = row -// val b: Option[(Long, (Product, (Map[String, Any], Map[String, Any])))] = info.get(TimeStampInfo.key) match { -// case Some(t: Long) => Some((t, row)) -// case _ => None +// val cacheResultProcesser = CacheResultProcesser() +// +// // init data stream +// sourceDataConnector.init() +// targetDataConnector.init() +// +// val streamingAccuracyProcess = StreamingAccuracyProcess( +// sourceDataConnector, targetDataConnector, +// ruleAnalyzer, cacheResultProcesser, persistFactory, appPersist) +// +// // process thread +//// case class Process() extends Runnable { +//// val lock = InfoCacheInstance.genLock("process") +//// def run(): Unit = { +//// val updateTime = new Date().getTime +//// val locked = lock.lock(5, TimeUnit.SECONDS) +//// if (locked) { +//// try { +//// val st = new Date().getTime +//// +//// TimeInfoCache.startTimeInfoCache +//// +//// // get data +//// val sourceData = sourceDataConnector.data match { +//// case Success(dt) => dt +//// case Failure(ex) => throw ex +//// } +//// val targetData = targetDataConnector.data match { +//// case Success(dt) => dt +//// case Failure(ex) => throw ex +//// } +//// +//// sourceData.cache +//// targetData.cache +//// +//// println(s"sourceData.count: ${sourceData.count}") +//// println(s"targetData.count: ${targetData.count}") +//// +//// // accuracy algorithm +//// val (accuResult, missingRdd, matchedRdd) = accuracy(sourceData, targetData, ruleAnalyzer) +//// println(s"accuResult: ${accuResult}") +//// +//// val ct = new Date().getTime +//// appPersist.log(ct, s"calculation using time: ${ct - st} ms") +//// +//// sourceData.unpersist() +//// targetData.unpersist() +//// +//// // result of every group +//// val matchedGroups = reorgByTimeGroup(matchedRdd) +//// val matchedGroupCount = matchedGroups.count +//// println(s"===== matchedGroupCount: ${matchedGroupCount} =====") +//// +//// // get missing results +//// val missingGroups = reorgByTimeGroup(missingRdd) +//// val missingGroupCount = missingGroups.count +//// println(s"===== missingGroupCount: ${missingGroupCount} =====") +//// +//// val groups = matchedGroups.cogroup(missingGroups) +//// val groupCount = groups.count +//// println(s"===== groupCount: ${groupCount} =====") +//// +//// val updateResults = groups.flatMap { group => +//// val (t, (matchData, missData)) = group +//// +//// val matchSize = matchData.size +//// val missSize = missData.size +//// val res = AccuracyResult(missSize, matchSize + missSize) +//// +//// val updatedCacheResultOpt = cacheResultProcesser.genUpdateCacheResult(t, updateTime, res) +//// +//// updatedCacheResultOpt.flatMap { updatedCacheResult => +//// Some((updatedCacheResult, (t, missData))) +//// } +//// } +//// +//// updateResults.cache +//// +//// val updateResultsPart = updateResults.map(_._1) +//// val updateDataPart = updateResults.map(_._2) +//// +//// val updateResultsArray = updateResultsPart.collect() +//// +//// // update results cache (in driver) +//// // collect action is traversable once action, it will make rdd updateResults empty +//// updateResultsArray.foreach { updateResult => +//// println(s"update result: ${updateResult}") +//// cacheResultProcesser.update(updateResult) +//// // persist result +//// val persist: Persist = persistFactory.getPersists(updateResult.timeGroup) +//// persist.result(updateTime, updateResult.result) +//// } +//// +//// // record missing data and update old data (in executor) +//// updateDataPart.foreach { grp => +//// val (t, datas) = grp +//// val persist: Persist = persistFactory.getPersists(t) +//// // persist missing data +//// val missStrings = datas.map { row => +//// val (_, (value, info)) = row +//// s"${value} [${info.getOrElse(MismatchInfo.key, "unknown")}]" +//// } +//// persist.records(missStrings, PersistType.MISS) +//// // data connector update old data +//// val dumpDatas = datas.map { r => +//// val (_, (v, i)) = r +//// v ++ i +//// } +//// +//// println(t) +//// dumpDatas.foreach(println) +//// +//// sourceDataConnector.updateOldData(t, dumpDatas) +//// targetDataConnector.updateOldData(t, dumpDatas) // not correct +//// } +//// +//// updateResults.unpersist() +//// +//// // dump missing rdd (this part not need for future version, only for current df cache data version) +//// val dumpRdd: RDD[Map[String, Any]] = missingRdd.map { r => +//// val (_, (v, i)) = r +//// v ++ i +//// } +//// sourceDataConnector.updateAllOldData(dumpRdd) +//// targetDataConnector.updateAllOldData(dumpRdd) // not correct +//// +//// TimeInfoCache.endTimeInfoCache +//// +//// val et = new Date().getTime +//// appPersist.log(et, s"persist using time: ${et - ct} ms") +//// +//// } catch { +//// case e: Throwable => error(s"process error: ${e.getMessage}") +//// } finally { +//// lock.unlock() +//// } +//// } +//// } +//// } +// +// val processInterval = TimeUtil.milliseconds(sparkParam.processInterval) match { +// case Some(interval) => interval +// case _ => throw new Exception("invalid batch interval") // } -// b +// val process = TimingProcess(processInterval, streamingAccuracyProcess) +// +// // clean thread +//// case class Clean() extends Runnable { +//// val lock = InfoCacheInstance.genLock("clean") +//// def run(): Unit = { +//// val locked = lock.lock(5, TimeUnit.SECONDS) +//// if (locked) { +//// try { +//// sourceDataConnector.cleanData +//// targetDataConnector.cleanData +//// } finally { +//// lock.unlock() +//// } +//// } +//// } +//// } +//// val cleanInterval = TimeUtil.milliseconds(cleanerParam.cleanInterval) match { +//// case Some(interval) => interval +//// case _ => throw new Exception("invalid batch interval") +//// } +//// val clean = TimingProcess(cleanInterval, Clean()) +// +// process.startup() +//// clean.startup() +// +// ssc.start() +// ssc.awaitTermination() +// ssc.stop(stopSparkContext=true, stopGracefully=true) +// +// // context stop +// sc.stop +// +// InfoCacheInstance.close +// +// appPersist.finish() +// +// process.shutdown() +//// clean.shutdown() // } // } - -} +// +// // calculate accuracy between source data and target data +//// def accuracy(sourceData: RDD[(Product, (Map[String, Any], Map[String, Any]))], +//// targetData: RDD[(Product, (Map[String, Any], Map[String, Any]))], +//// ruleAnalyzer: RuleAnalyzer) = { +//// // 1. cogroup +//// val allKvs = sourceData.cogroup(targetData) +//// +//// // 2. accuracy calculation +//// val (accuResult, missingRdd, matchedRdd) = AccuracyCore.accuracy(allKvs, ruleAnalyzer) +//// +//// (accuResult, missingRdd, matchedRdd) +//// } +// +//// // convert data into a string +//// def record2String(rec: (Product, (Map[String, Any], Map[String, Any])), sourcePersist: Iterable[Expr], targetPersist: Iterable[Expr]): String = { +//// val (key, (data, info)) = rec +//// val persistData = getPersistMap(data, sourcePersist) +//// val persistInfo = info.mapValues { value => +//// value match { +//// case vd: Map[String, Any] => getPersistMap(vd, targetPersist) +//// case v => v +//// } +//// }.map(identity) +//// s"${persistData} [${persistInfo}]" +//// } +//// +//// // get the expr value map of the persist expressions +//// private def getPersistMap(data: Map[String, Any], persist: Iterable[Expr]): Map[String, Any] = { +//// val persistMap = persist.map(e => (e._id, e.desc)).toMap +//// data.flatMap { pair => +//// val (k, v) = pair +//// persistMap.get(k) match { +//// case Some(d) => Some((d -> v)) +//// case _ => None +//// } +//// } +//// } +// +//// def reorgByTimeGroup(rdd: RDD[(Product, (Map[String, Any], Map[String, Any]))] +//// ): RDD[(Long, (Product, (Map[String, Any], Map[String, Any])))] = { +//// rdd.flatMap { row => +//// val (key, (value, info)) = row +//// val b: Option[(Long, (Product, (Map[String, Any], Map[String, Any])))] = info.get(TimeStampInfo.key) match { +//// case Some(t: Long) => Some((t, row)) +//// case _ => None +//// } +//// b +//// } +//// } +// +//} diff --git a/measure/src/main/scala/org/apache/griffin/measure/config/params/user/DataSourceParam.scala b/measure/src/main/scala/org/apache/griffin/measure/config/params/user/DataSourceParam.scala new file mode 100644 index 000000000..48ba81a97 --- /dev/null +++ b/measure/src/main/scala/org/apache/griffin/measure/config/params/user/DataSourceParam.scala @@ -0,0 +1,30 @@ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ +package org.apache.griffin.measure.config.params.user + +import com.fasterxml.jackson.annotation.{JsonInclude, JsonProperty} +import com.fasterxml.jackson.annotation.JsonInclude.Include +import org.apache.griffin.measure.config.params.Param + +@JsonInclude(Include.NON_NULL) +case class DataSourceParam( @JsonProperty("name") name: String, + @JsonProperty("connectors") connectors: List[DataConnectorParam] + ) extends Param { + +} diff --git a/measure/src/main/scala/org/apache/griffin/measure/config/params/user/EvaluateRuleParam.scala b/measure/src/main/scala/org/apache/griffin/measure/config/params/user/EvaluateRuleParam.scala index 6ee978344..eddb435cc 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/config/params/user/EvaluateRuleParam.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/config/params/user/EvaluateRuleParam.scala @@ -23,8 +23,8 @@ import com.fasterxml.jackson.annotation.JsonInclude.Include import org.apache.griffin.measure.config.params.Param @JsonInclude(Include.NON_NULL) -case class EvaluateRuleParam( @JsonProperty("sampleRatio") sampleRatio: Double, - @JsonProperty("rules") rules: String +case class EvaluateRuleParam( @JsonProperty("dsl.type") dslType: String, + @JsonProperty("rules") rules: List[RuleParam] ) extends Param { } diff --git a/measure/src/main/scala/org/apache/griffin/measure/config/params/user/RuleParam.scala b/measure/src/main/scala/org/apache/griffin/measure/config/params/user/RuleParam.scala new file mode 100644 index 000000000..6ce2ac162 --- /dev/null +++ b/measure/src/main/scala/org/apache/griffin/measure/config/params/user/RuleParam.scala @@ -0,0 +1,32 @@ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ +package org.apache.griffin.measure.config.params.user + +import com.fasterxml.jackson.annotation.{JsonInclude, JsonProperty} +import com.fasterxml.jackson.annotation.JsonInclude.Include +import org.apache.griffin.measure.config.params.Param + +@JsonInclude(Include.NON_NULL) +case class RuleParam( @JsonProperty("name") name: String, + @JsonProperty("type") ruleType: String, + @JsonProperty("rule") procType: String, + @JsonProperty("persist.type") persistType: String + ) extends Param { + +} diff --git a/measure/src/main/scala/org/apache/griffin/measure/config/params/user/UserParam.scala b/measure/src/main/scala/org/apache/griffin/measure/config/params/user/UserParam.scala index df0647cc5..e55d2b40c 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/config/params/user/UserParam.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/config/params/user/UserParam.scala @@ -23,12 +23,10 @@ import com.fasterxml.jackson.annotation.JsonInclude.Include import org.apache.griffin.measure.config.params.Param @JsonInclude(Include.NON_NULL) -case class UserParam(@JsonProperty("name") name: String, - @JsonProperty("type") dqType: String, - @JsonProperty("process.type") procType: String, - @JsonProperty("source") sourceParam: DataConnectorParam, - @JsonProperty("target") targetParam: DataConnectorParam, - @JsonProperty("evaluateRule") evaluateRuleParam: EvaluateRuleParam +case class UserParam( @JsonProperty("name") name: String, + @JsonProperty("process.type") procType: String, + @JsonProperty("data.sources") dataSources: List[DataSourceParam], + @JsonProperty("evaluateRule") evaluateRuleParam: EvaluateRuleParam ) extends Param { } diff --git a/measure/src/main/scala/org/apache/griffin/measure/rule/RuleFactory.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/RuleFactory.scala index bbaf5cb29..d0120498f 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rule/RuleFactory.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rule/RuleFactory.scala @@ -1,52 +1,52 @@ -/* -Licensed to the Apache Software Foundation (ASF) under one -or more contributor license agreements. See the NOTICE file -distributed with this work for additional information -regarding copyright ownership. The ASF licenses this file -to you under the Apache License, Version 2.0 (the -"License"); you may not use this file except in compliance -with the License. You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, -software distributed under the License is distributed on an -"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -KIND, either express or implied. See the License for the -specific language governing permissions and limitations -under the License. -*/ -package org.apache.griffin.measure.rule - -import org.apache.griffin.measure.config.params.user._ - -import scala.util.Failure -//import org.apache.griffin.measure.rule.expr_old._ -import org.apache.griffin.measure.rule.expr._ - -import scala.util.{Success, Try} - - -case class RuleFactory(evaluateRuleParam: EvaluateRuleParam) { - - val ruleParser: RuleParser = RuleParser() - - def generateRule(): StatementExpr = { - val rules = evaluateRuleParam.rules - val statement = parseExpr(rules) match { - case Success(se) => se - case Failure(ex) => throw ex - } - statement - } - - private def parseExpr(rules: String): Try[StatementExpr] = { - Try { - val result = ruleParser.parseAll(ruleParser.rule, rules) - if (result.successful) result.get - else throw new Exception("parse rule error!") -// throw new Exception("parse rule error!") - } - } - -} +///* +//Licensed to the Apache Software Foundation (ASF) under one +//or more contributor license agreements. See the NOTICE file +//distributed with this work for additional information +//regarding copyright ownership. The ASF licenses this file +//to you under the Apache License, Version 2.0 (the +//"License"); you may not use this file except in compliance +//with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +//Unless required by applicable law or agreed to in writing, +//software distributed under the License is distributed on an +//"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +//KIND, either express or implied. See the License for the +//specific language governing permissions and limitations +//under the License. +//*/ +//package org.apache.griffin.measure.rule +// +//import org.apache.griffin.measure.config.params.user._ +// +//import scala.util.Failure +////import org.apache.griffin.measure.rule.expr_old._ +//import org.apache.griffin.measure.rule.expr._ +// +//import scala.util.{Success, Try} +// +// +//case class RuleFactory(evaluateRuleParam: EvaluateRuleParam) { +// +// val ruleParser: RuleParser = RuleParser() +// +// def generateRule(): StatementExpr = { +// val rules = evaluateRuleParam.rules +// val statement = parseExpr(rules) match { +// case Success(se) => se +// case Failure(ex) => throw ex +// } +// statement +// } +// +// private def parseExpr(rules: String): Try[StatementExpr] = { +// Try { +// val result = ruleParser.parseAll(ruleParser.rule, rules) +// if (result.successful) result.get +// else throw new Exception("parse rule error!") +//// throw new Exception("parse rule error!") +// } +// } +// +//} diff --git a/measure/src/test/scala/org/apache/griffin/measure/algo/batch/BatchAccuracyAlgoTest.scala b/measure/src/test/scala/org/apache/griffin/measure/algo/batch/BatchAccuracyAlgoTest.scala index 0d26411dc..6615d1adf 100644 --- a/measure/src/test/scala/org/apache/griffin/measure/algo/batch/BatchAccuracyAlgoTest.scala +++ b/measure/src/test/scala/org/apache/griffin/measure/algo/batch/BatchAccuracyAlgoTest.scala @@ -1,294 +1,294 @@ -/* -Licensed to the Apache Software Foundation (ASF) under one -or more contributor license agreements. See the NOTICE file -distributed with this work for additional information -regarding copyright ownership. The ASF licenses this file -to you under the Apache License, Version 2.0 (the -"License"); you may not use this file except in compliance -with the License. You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, -software distributed under the License is distributed on an -"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -KIND, either express or implied. See the License for the -specific language governing permissions and limitations -under the License. -*/ -package org.apache.griffin.measure.algo - -import java.util.Date - -import org.apache.griffin.measure.algo.batch.BatchAccuracyAlgo -import org.apache.griffin.measure.config.params._ -import org.apache.griffin.measure.config.params.env._ -import org.apache.griffin.measure.config.params.user._ -import org.apache.griffin.measure.config.reader._ -import org.apache.griffin.measure.config.validator._ -import org.apache.griffin.measure.connector.direct.DirectDataConnector -import org.apache.griffin.measure.connector.{DataConnector, DataConnectorFactory} -import org.apache.griffin.measure.log.Loggable -import org.apache.griffin.measure.rule.expr._ -import org.apache.griffin.measure.rule.{ExprValueUtil, RuleAnalyzer, RuleFactory} -import org.apache.spark.rdd.RDD -import org.apache.spark.sql.SQLContext -import org.apache.spark.sql.hive.HiveContext -import org.apache.spark.{SparkConf, SparkContext} -import org.junit.runner.RunWith -import org.scalatest.junit.JUnitRunner -import org.scalatest.{BeforeAndAfter, FunSuite, Matchers} - -import scala.util.{Failure, Success, Try} - - -@RunWith(classOf[JUnitRunner]) -class BatchAccuracyAlgoTest extends FunSuite with Matchers with BeforeAndAfter with Loggable { - - val envFile = "src/test/resources/env.json" - val confFile = "src/test/resources/config.json" -// val confFile = "{\"name\":\"accu1\",\"type\":\"accuracy\",\"source\":{\"type\":\"avro\",\"version\":\"1.7\",\"config\":{\"file.name\":\"src/test/resources/users_info_src.avro\"}},\"target\":{\"type\":\"avro\",\"version\":\"1.7\",\"config\":{\"file.name\":\"src/test/resources/users_info_target.avro\"}},\"evaluateRule\":{\"sampleRatio\":1,\"rules\":\"$source.user_id + 5 = $target.user_id + (2 + 3) AND $source.first_name + 12 = $target.first_name + (10 + 2) 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 AND (15 OR true) WHEN true AND $source.user_id > 10020\"}}" - val envFsType = "local" - val userFsType = "local" - - val args = Array(envFile, confFile) - - var sc: SparkContext = _ - var sqlContext: SQLContext = _ - - var allParam: AllParam = _ - - before { - // read param files - val envParam = readParamFile[EnvParam](envFile, envFsType) match { - case Success(p) => p - case Failure(ex) => { - error(ex.getMessage) - sys.exit(-2) - } - } - val userParam = readParamFile[UserParam](confFile, userFsType) match { - case Success(p) => p - case Failure(ex) => { - error(ex.getMessage) - sys.exit(-2) - } - } - allParam = AllParam(envParam, userParam) - - // validate param files - validateParams(allParam) match { - case Failure(ex) => { - error(ex.getMessage) - sys.exit(-3) - } - case _ => { - info("params validation pass") - } - } - - val metricName = userParam.name - val conf = new SparkConf().setMaster("local[*]").setAppName(metricName) - sc = new SparkContext(conf) -// sqlContext = new SQLContext(sc) - sqlContext = new HiveContext(sc) - } - - test("algorithm") { - Try { - val envParam = allParam.envParam - val userParam = allParam.userParam - - // start time - val startTime = new Date().getTime() - - // get spark application id - val applicationId = sc.applicationId - - // rules - val ruleFactory = RuleFactory(userParam.evaluateRuleParam) - val rule: StatementExpr = ruleFactory.generateRule() - val ruleAnalyzer: RuleAnalyzer = RuleAnalyzer(rule) - - ruleAnalyzer.constCacheExprs.foreach(println) - ruleAnalyzer.constFinalCacheExprs.foreach(println) - - // global cache data - val constExprValueMap = ExprValueUtil.genExprValueMaps(None, ruleAnalyzer.constCacheExprs, Map[String, Any]()) - val finalConstExprValueMap = ExprValueUtil.updateExprValueMaps(ruleAnalyzer.constFinalCacheExprs, constExprValueMap) - val finalConstMap = finalConstExprValueMap.headOption match { - case Some(m) => m - case _ => Map[String, Any]() - } - - // data connector - val sourceDataConnector: DirectDataConnector = - DataConnectorFactory.getDirectDataConnector(sqlContext, null, userParam.sourceParam, - ruleAnalyzer.sourceRuleExprs, finalConstMap - ) match { - case Success(cntr) => { - if (cntr.available) cntr - else throw new Exception("source data not available!") - } - case Failure(ex) => throw ex - } - val targetDataConnector: DirectDataConnector = - DataConnectorFactory.getDirectDataConnector(sqlContext, null, userParam.targetParam, - ruleAnalyzer.targetRuleExprs, finalConstMap - ) match { - case Success(cntr) => { - if (cntr.available) cntr - else throw new Exception("target data not available!") - } - case Failure(ex) => throw ex - } - - // get metadata -// val sourceMetaData: Iterable[(String, String)] = sourceDataConnector.metaData() match { -// case Success(md) => md +///* +//Licensed to the Apache Software Foundation (ASF) under one +//or more contributor license agreements. See the NOTICE file +//distributed with this work for additional information +//regarding copyright ownership. The ASF licenses this file +//to you under the Apache License, Version 2.0 (the +//"License"); you may not use this file except in compliance +//with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +//Unless required by applicable law or agreed to in writing, +//software distributed under the License is distributed on an +//"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +//KIND, either express or implied. See the License for the +//specific language governing permissions and limitations +//under the License. +//*/ +//package org.apache.griffin.measure.algo +// +//import java.util.Date +// +//import org.apache.griffin.measure.algo.batch.BatchAccuracyAlgo +//import org.apache.griffin.measure.config.params._ +//import org.apache.griffin.measure.config.params.env._ +//import org.apache.griffin.measure.config.params.user._ +//import org.apache.griffin.measure.config.reader._ +//import org.apache.griffin.measure.config.validator._ +//import org.apache.griffin.measure.connector.direct.DirectDataConnector +//import org.apache.griffin.measure.connector.{DataConnector, DataConnectorFactory} +//import org.apache.griffin.measure.log.Loggable +//import org.apache.griffin.measure.rule.expr._ +//import org.apache.griffin.measure.rule.{ExprValueUtil, RuleAnalyzer, RuleFactory} +//import org.apache.spark.rdd.RDD +//import org.apache.spark.sql.SQLContext +//import org.apache.spark.sql.hive.HiveContext +//import org.apache.spark.{SparkConf, SparkContext} +//import org.junit.runner.RunWith +//import org.scalatest.junit.JUnitRunner +//import org.scalatest.{BeforeAndAfter, FunSuite, Matchers} +// +//import scala.util.{Failure, Success, Try} +// +// +//@RunWith(classOf[JUnitRunner]) +//class BatchAccuracyAlgoTest extends FunSuite with Matchers with BeforeAndAfter with Loggable { +// +// val envFile = "src/test/resources/env.json" +// val confFile = "src/test/resources/config.json" +//// val confFile = "{\"name\":\"accu1\",\"type\":\"accuracy\",\"source\":{\"type\":\"avro\",\"version\":\"1.7\",\"config\":{\"file.name\":\"src/test/resources/users_info_src.avro\"}},\"target\":{\"type\":\"avro\",\"version\":\"1.7\",\"config\":{\"file.name\":\"src/test/resources/users_info_target.avro\"}},\"evaluateRule\":{\"sampleRatio\":1,\"rules\":\"$source.user_id + 5 = $target.user_id + (2 + 3) AND $source.first_name + 12 = $target.first_name + (10 + 2) 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 AND (15 OR true) WHEN true AND $source.user_id > 10020\"}}" +// val envFsType = "local" +// val userFsType = "local" +// +// val args = Array(envFile, confFile) +// +// var sc: SparkContext = _ +// var sqlContext: SQLContext = _ +// +// var allParam: AllParam = _ +// +// before { +// // read param files +// val envParam = readParamFile[EnvParam](envFile, envFsType) match { +// case Success(p) => p +// case Failure(ex) => { +// error(ex.getMessage) +// sys.exit(-2) +// } +// } +// val userParam = readParamFile[UserParam](confFile, userFsType) match { +// case Success(p) => p +// case Failure(ex) => { +// error(ex.getMessage) +// sys.exit(-2) +// } +// } +// allParam = AllParam(envParam, userParam) +// +// // validate param files +// validateParams(allParam) match { +// case Failure(ex) => { +// error(ex.getMessage) +// sys.exit(-3) +// } +// case _ => { +// info("params validation pass") +// } +// } +// +// val metricName = userParam.name +// val conf = new SparkConf().setMaster("local[*]").setAppName(metricName) +// sc = new SparkContext(conf) +//// sqlContext = new SQLContext(sc) +// sqlContext = new HiveContext(sc) +// } +// +// test("algorithm") { +// Try { +// val envParam = allParam.envParam +// val userParam = allParam.userParam +// +// // start time +// val startTime = new Date().getTime() +// +// // get spark application id +// val applicationId = sc.applicationId +// +// // rules +// val ruleFactory = RuleFactory(userParam.evaluateRuleParam) +// val rule: StatementExpr = ruleFactory.generateRule() +// val ruleAnalyzer: RuleAnalyzer = RuleAnalyzer(rule) +// +// ruleAnalyzer.constCacheExprs.foreach(println) +// ruleAnalyzer.constFinalCacheExprs.foreach(println) +// +// // global cache data +// val constExprValueMap = ExprValueUtil.genExprValueMaps(None, ruleAnalyzer.constCacheExprs, Map[String, Any]()) +// val finalConstExprValueMap = ExprValueUtil.updateExprValueMaps(ruleAnalyzer.constFinalCacheExprs, constExprValueMap) +// val finalConstMap = finalConstExprValueMap.headOption match { +// case Some(m) => m +// case _ => Map[String, Any]() +// } +// +// // data connector +// val sourceDataConnector: DirectDataConnector = +// DataConnectorFactory.getDirectDataConnector(sqlContext, null, userParam.sourceParam, +// ruleAnalyzer.sourceRuleExprs, finalConstMap +// ) match { +// case Success(cntr) => { +// if (cntr.available) cntr +// else throw new Exception("source data not available!") +// } +// case Failure(ex) => throw ex +// } +// val targetDataConnector: DirectDataConnector = +// DataConnectorFactory.getDirectDataConnector(sqlContext, null, userParam.targetParam, +// ruleAnalyzer.targetRuleExprs, finalConstMap +// ) match { +// case Success(cntr) => { +// if (cntr.available) cntr +// else throw new Exception("target data not available!") +// } +// case Failure(ex) => throw ex +// } +// +// // get metadata +//// val sourceMetaData: Iterable[(String, String)] = sourceDataConnector.metaData() match { +//// case Success(md) => md +//// case Failure(ex) => throw ex +//// } +//// val targetMetaData: Iterable[(String, String)] = targetDataConnector.metaData() match { +//// case Success(md) => md +//// case Failure(ex) => throw ex +//// } +// +// // get data +// val sourceData: RDD[(Product, (Map[String, Any], Map[String, Any]))] = sourceDataConnector.data() match { +// case Success(dt) => dt // case Failure(ex) => throw ex // } -// val targetMetaData: Iterable[(String, String)] = targetDataConnector.metaData() match { -// case Success(md) => md +// val targetData: RDD[(Product, (Map[String, Any], Map[String, Any]))] = targetDataConnector.data() match { +// case Success(dt) => dt // case Failure(ex) => throw ex // } - - // get data - val sourceData: RDD[(Product, (Map[String, Any], Map[String, Any]))] = sourceDataConnector.data() match { - case Success(dt) => dt - case Failure(ex) => throw ex - } - val targetData: RDD[(Product, (Map[String, Any], Map[String, Any]))] = targetDataConnector.data() match { - case Success(dt) => dt - case Failure(ex) => throw ex - } - - // my algo - val algo = BatchAccuracyAlgo(allParam) - - // accuracy algorithm - val (accuResult, missingRdd, matchedRdd) = algo.accuracy(sourceData, targetData, ruleAnalyzer) - - println(s"match percentage: ${accuResult.matchPercentage}, total count: ${accuResult.total}") - - missingRdd.map(rec => algo.record2String(rec, ruleAnalyzer.sourceRuleExprs.persistExprs, ruleAnalyzer.targetRuleExprs.persistExprs)).foreach(println) - - // end time - val endTime = new Date().getTime - println(s"using time: ${endTime - startTime} ms") - } match { - case Failure(ex) => { - error(ex.getMessage) - sys.exit(-4) - } - case _ => { - info("calculation finished") - } - } - } - - private def readParamFile[T <: Param](file: String, fsType: String)(implicit m : Manifest[T]): Try[T] = { - val paramReader = ParamReaderFactory.getParamReader(file, fsType) - paramReader.readConfig[T] - } - - private def validateParams(allParam: AllParam): Try[Boolean] = { - val allParamValidator = AllParamValidator() - allParamValidator.validate(allParam) - } - - test ("spark sql") { - Try { - val envParam = allParam.envParam - val userParam = allParam.userParam - - // start time - val startTime = new Date().getTime() - - // get spark application id - val applicationId = sc.applicationId - -// val sourceFilePath = "src/test/resources/users_info_src.avro" -// val targetFilePath = "src/test/resources/users_info_target.avro" -// -// val sourceDF = sqlContext.read.format("com.databricks.spark.avro").load(sourceFilePath) -// val targetDF = sqlContext.read.format("com.databricks.spark.avro").load(targetFilePath) - - val sourceTableName = "source.table" - val targetTableName = "target.table" - - val sourceDF = sqlContext.sql("SELECT * FROM default.data_avr") - val targetDF = sqlContext.sql("SELECT * FROM default.data_only") - -// sourceDF.show(100) -// targetDF.show(100) - - sourceDF.registerTempTable(sourceTableName) - targetDF.registerTempTable(targetTableName) - -// val sourceTableName = "data_avr" -// val targetTableName = "data_avr" - +// +// // my algo +// val algo = BatchAccuracyAlgo(allParam) +// +// // accuracy algorithm +// val (accuResult, missingRdd, matchedRdd) = algo.accuracy(sourceData, targetData, ruleAnalyzer) +// +// println(s"match percentage: ${accuResult.matchPercentage}, total count: ${accuResult.total}") +// +// missingRdd.map(rec => algo.record2String(rec, ruleAnalyzer.sourceRuleExprs.persistExprs, ruleAnalyzer.targetRuleExprs.persistExprs)).foreach(println) +// +// // end time +// val endTime = new Date().getTime +// println(s"using time: ${endTime - startTime} ms") +// } match { +// case Failure(ex) => { +// error(ex.getMessage) +// sys.exit(-4) +// } +// case _ => { +// info("calculation finished") +// } +// } +// } +// +// private def readParamFile[T <: Param](file: String, fsType: String)(implicit m : Manifest[T]): Try[T] = { +// val paramReader = ParamReaderFactory.getParamReader(file, fsType) +// paramReader.readConfig[T] +// } +// +// private def validateParams(allParam: AllParam): Try[Boolean] = { +// val allParamValidator = AllParamValidator() +// allParamValidator.validate(allParam) +// } +// +// test ("spark sql") { +// Try { +// val envParam = allParam.envParam +// val userParam = allParam.userParam +// +// // start time +// val startTime = new Date().getTime() +// +// // get spark application id +// val applicationId = sc.applicationId +// +//// val sourceFilePath = "src/test/resources/users_info_src.avro" +//// val targetFilePath = "src/test/resources/users_info_target.avro" +//// +//// val sourceDF = sqlContext.read.format("com.databricks.spark.avro").load(sourceFilePath) +//// val targetDF = sqlContext.read.format("com.databricks.spark.avro").load(targetFilePath) +// +// val sourceTableName = "source.table" +// val targetTableName = "target.table" +// +// val sourceDF = sqlContext.sql("SELECT * FROM default.data_avr") +// val targetDF = sqlContext.sql("SELECT * FROM default.data_only") +// +//// sourceDF.show(100) +//// targetDF.show(100) +// +// sourceDF.registerTempTable(sourceTableName) +// targetDF.registerTempTable(targetTableName) +// +//// val sourceTableName = "data_avr" +//// val targetTableName = "data_avr" +// +//// val sql = +//// s""" +//// |SELECT COUNT(*) FROM `${sourceTableName}` LEFT JOIN `${targetTableName}` +//// |ON `${sourceTableName}`.uid = `${targetTableName}`.uid +//// """.stripMargin +// // val sql = // s""" -// |SELECT COUNT(*) FROM `${sourceTableName}` LEFT JOIN `${targetTableName}` -// |ON `${sourceTableName}`.uid = `${targetTableName}`.uid -// """.stripMargin - - val sql = - s""" - |SELECT `${sourceTableName}`.uid, `${sourceTableName}`.uage, `${sourceTableName}`.udes, - |`${targetTableName}`.uid, `${targetTableName}`.uage, `${targetTableName}`.udes - |FROM `${sourceTableName}` LEFT JOIN `${targetTableName}` - |ON coalesce(`${sourceTableName}`.uid, 'null') = coalesce(`${targetTableName}`.uid, 'null') - |AND coalesce(`${sourceTableName}`.uage, 'null') = coalesce(`${targetTableName}`.uage, 'null') - |AND coalesce(`${sourceTableName}`.udes, 'null') = coalesce(`${targetTableName}`.udes, 'null') - |WHERE (NOT (`${sourceTableName}`.uid IS NULL - |AND `${sourceTableName}`.uage IS NULL - |AND `${sourceTableName}`.udes IS NULL)) - |AND ((`${targetTableName}`.uid IS NULL - |AND `${targetTableName}`.uage IS NULL - |AND `${targetTableName}`.udes IS NULL)) - """.stripMargin - -// val sql = -// """ -// |SELECT * FROM source LEFT JOIN target -// |ON source.user_id = target.user_id -// |AND source.first_name = 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 coalesce(source.post_code, 'null') = coalesce(target.post_code, 'null') +// |SELECT `${sourceTableName}`.uid, `${sourceTableName}`.uage, `${sourceTableName}`.udes, +// |`${targetTableName}`.uid, `${targetTableName}`.uage, `${targetTableName}`.udes +// |FROM `${sourceTableName}` LEFT JOIN `${targetTableName}` +// |ON coalesce(`${sourceTableName}`.uid, 'null') = coalesce(`${targetTableName}`.uid, 'null') +// |AND coalesce(`${sourceTableName}`.uage, 'null') = coalesce(`${targetTableName}`.uage, 'null') +// |AND coalesce(`${sourceTableName}`.udes, 'null') = coalesce(`${targetTableName}`.udes, 'null') +// |WHERE (NOT (`${sourceTableName}`.uid IS NULL +// |AND `${sourceTableName}`.uage IS NULL +// |AND `${sourceTableName}`.udes IS NULL)) +// |AND ((`${targetTableName}`.uid IS NULL +// |AND `${targetTableName}`.uage IS NULL +// |AND `${targetTableName}`.udes IS NULL)) // """.stripMargin - -// val sql = -// """ -// |SELECT * FROM source WHERE source.post_code IS NULL -// """.stripMargin - - val result = sqlContext.sql(sql) - - result.show(100) - -// result.registerTempTable("result") -// val rsql = "SELECT COUNT(*) FROM result" -// val rr = sqlContext.sql(rsql) -// rr.show(100) - - // end time - val endTime = new Date().getTime - println(s"using time: ${endTime - startTime} ms") - } match { - case Failure(ex) => { - error(ex.getMessage) - sys.exit(-4) - } - case _ => { - info("calculation finished") - } - } - } - -} +// +//// val sql = +//// """ +//// |SELECT * FROM source LEFT JOIN target +//// |ON source.user_id = target.user_id +//// |AND source.first_name = 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 coalesce(source.post_code, 'null') = coalesce(target.post_code, 'null') +//// """.stripMargin +// +//// val sql = +//// """ +//// |SELECT * FROM source WHERE source.post_code IS NULL +//// """.stripMargin +// +// val result = sqlContext.sql(sql) +// +// result.show(100) +// +//// result.registerTempTable("result") +//// val rsql = "SELECT COUNT(*) FROM result" +//// val rr = sqlContext.sql(rsql) +//// rr.show(100) +// +// // end time +// val endTime = new Date().getTime +// println(s"using time: ${endTime - startTime} ms") +// } match { +// case Failure(ex) => { +// error(ex.getMessage) +// sys.exit(-4) +// } +// case _ => { +// info("calculation finished") +// } +// } +// } +// +//} diff --git a/measure/src/test/scala/org/apache/griffin/measure/algo/core/AccuracyCoreTest.scala b/measure/src/test/scala/org/apache/griffin/measure/algo/core/AccuracyCoreTest.scala index 2179fbabf..034add69b 100644 --- a/measure/src/test/scala/org/apache/griffin/measure/algo/core/AccuracyCoreTest.scala +++ b/measure/src/test/scala/org/apache/griffin/measure/algo/core/AccuracyCoreTest.scala @@ -1,89 +1,89 @@ -/* -Licensed to the Apache Software Foundation (ASF) under one -or more contributor license agreements. See the NOTICE file -distributed with this work for additional information -regarding copyright ownership. The ASF licenses this file -to you under the Apache License, Version 2.0 (the -"License"); you may not use this file except in compliance -with the License. You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, -software distributed under the License is distributed on an -"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -KIND, either express or implied. See the License for the -specific language governing permissions and limitations -under the License. -*/ -package org.apache.griffin.measure.algo.core - -import org.apache.griffin.measure.config.params.user.EvaluateRuleParam -import org.apache.griffin.measure.rule.expr._ -import org.apache.griffin.measure.rule.{RuleAnalyzer, RuleFactory} -import org.junit.runner.RunWith -import org.scalatest.junit.JUnitRunner -import org.scalatest.{BeforeAndAfter, FunSuite, Matchers} -import org.scalatest.PrivateMethodTester - -@RunWith(classOf[JUnitRunner]) -class AccuracyCoreTest extends FunSuite with Matchers with BeforeAndAfter with PrivateMethodTester { - - def findExprId(exprs: Iterable[Expr], desc: String): String = { - exprs.find(_.desc == desc) match { - case Some(expr) => expr._id - case _ => "" - } - } - - test ("match data success") { - val rule = "$source.name = $target.name AND $source.age < $target.age" - val evaluateRuleParam = EvaluateRuleParam(1.0, rule) - val ruleFactory = RuleFactory(evaluateRuleParam) - val statement = ruleFactory.generateRule - val ruleAnalyzer = RuleAnalyzer(statement) - - val sourcePersistExprs = ruleAnalyzer.sourceRuleExprs.persistExprs - val targetPersistExprs = ruleAnalyzer.targetRuleExprs.persistExprs - - val source = (Map[String, Any]( - (findExprId(sourcePersistExprs, "$source['name']") -> "jack"), - (findExprId(sourcePersistExprs, "$source['age']") -> 26) - ), Map[String, Any]()) - val target = (Map[String, Any]( - (findExprId(targetPersistExprs, "$target['name']") -> "jack"), - (findExprId(targetPersistExprs, "$target['age']") -> 27) - ), Map[String, Any]()) - - val matchData = PrivateMethod[(Boolean, Map[String, Any])]('matchData) - val result = AccuracyCore invokePrivate matchData(source, target, ruleAnalyzer) - result._1 should be (true) - result._2.size should be (0) - } - - test ("match data fail") { - val rule = "$source.name = $target.name AND $source.age = $target.age" - val evaluateRuleParam = EvaluateRuleParam(1.0, rule) - val ruleFactory = RuleFactory(evaluateRuleParam) - val statement = ruleFactory.generateRule - val ruleAnalyzer = RuleAnalyzer(statement) - - val sourcePersistExprs = ruleAnalyzer.sourceRuleExprs.persistExprs - val targetPersistExprs = ruleAnalyzer.targetRuleExprs.persistExprs - - val source = (Map[String, Any]( - (findExprId(sourcePersistExprs, "$source['name']") -> "jack"), - (findExprId(sourcePersistExprs, "$source['age']") -> 26) - ), Map[String, Any]()) - val target = (Map[String, Any]( - (findExprId(targetPersistExprs, "$target['name']") -> "jack"), - (findExprId(targetPersistExprs, "$target['age']") -> 27) - ), Map[String, Any]()) - - val matchData = PrivateMethod[(Boolean, Map[String, Any])]('matchData) - val result = AccuracyCore invokePrivate matchData(source, target, ruleAnalyzer) - result._1 should be (false) - result._2.size shouldNot be (0) - } - -} +///* +//Licensed to the Apache Software Foundation (ASF) under one +//or more contributor license agreements. See the NOTICE file +//distributed with this work for additional information +//regarding copyright ownership. The ASF licenses this file +//to you under the Apache License, Version 2.0 (the +//"License"); you may not use this file except in compliance +//with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +//Unless required by applicable law or agreed to in writing, +//software distributed under the License is distributed on an +//"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +//KIND, either express or implied. See the License for the +//specific language governing permissions and limitations +//under the License. +//*/ +//package org.apache.griffin.measure.algo.core +// +//import org.apache.griffin.measure.config.params.user.EvaluateRuleParam +//import org.apache.griffin.measure.rule.expr._ +//import org.apache.griffin.measure.rule.{RuleAnalyzer, RuleFactory} +//import org.junit.runner.RunWith +//import org.scalatest.junit.JUnitRunner +//import org.scalatest.{BeforeAndAfter, FunSuite, Matchers} +//import org.scalatest.PrivateMethodTester +// +//@RunWith(classOf[JUnitRunner]) +//class AccuracyCoreTest extends FunSuite with Matchers with BeforeAndAfter with PrivateMethodTester { +// +// def findExprId(exprs: Iterable[Expr], desc: String): String = { +// exprs.find(_.desc == desc) match { +// case Some(expr) => expr._id +// case _ => "" +// } +// } +// +// test ("match data success") { +// val rule = "$source.name = $target.name AND $source.age < $target.age" +// val evaluateRuleParam = EvaluateRuleParam(1.0, rule) +// val ruleFactory = RuleFactory(evaluateRuleParam) +// val statement = ruleFactory.generateRule +// val ruleAnalyzer = RuleAnalyzer(statement) +// +// val sourcePersistExprs = ruleAnalyzer.sourceRuleExprs.persistExprs +// val targetPersistExprs = ruleAnalyzer.targetRuleExprs.persistExprs +// +// val source = (Map[String, Any]( +// (findExprId(sourcePersistExprs, "$source['name']") -> "jack"), +// (findExprId(sourcePersistExprs, "$source['age']") -> 26) +// ), Map[String, Any]()) +// val target = (Map[String, Any]( +// (findExprId(targetPersistExprs, "$target['name']") -> "jack"), +// (findExprId(targetPersistExprs, "$target['age']") -> 27) +// ), Map[String, Any]()) +// +// val matchData = PrivateMethod[(Boolean, Map[String, Any])]('matchData) +// val result = AccuracyCore invokePrivate matchData(source, target, ruleAnalyzer) +// result._1 should be (true) +// result._2.size should be (0) +// } +// +// test ("match data fail") { +// val rule = "$source.name = $target.name AND $source.age = $target.age" +// val evaluateRuleParam = EvaluateRuleParam(1.0, rule) +// val ruleFactory = RuleFactory(evaluateRuleParam) +// val statement = ruleFactory.generateRule +// val ruleAnalyzer = RuleAnalyzer(statement) +// +// val sourcePersistExprs = ruleAnalyzer.sourceRuleExprs.persistExprs +// val targetPersistExprs = ruleAnalyzer.targetRuleExprs.persistExprs +// +// val source = (Map[String, Any]( +// (findExprId(sourcePersistExprs, "$source['name']") -> "jack"), +// (findExprId(sourcePersistExprs, "$source['age']") -> 26) +// ), Map[String, Any]()) +// val target = (Map[String, Any]( +// (findExprId(targetPersistExprs, "$target['name']") -> "jack"), +// (findExprId(targetPersistExprs, "$target['age']") -> 27) +// ), Map[String, Any]()) +// +// val matchData = PrivateMethod[(Boolean, Map[String, Any])]('matchData) +// val result = AccuracyCore invokePrivate matchData(source, target, ruleAnalyzer) +// result._1 should be (false) +// result._2.size shouldNot be (0) +// } +// +//} diff --git a/measure/src/test/scala/org/apache/griffin/measure/algo/core/ProfileCoreTest.scala b/measure/src/test/scala/org/apache/griffin/measure/algo/core/ProfileCoreTest.scala index 087e8e59e..53b91c044 100644 --- a/measure/src/test/scala/org/apache/griffin/measure/algo/core/ProfileCoreTest.scala +++ b/measure/src/test/scala/org/apache/griffin/measure/algo/core/ProfileCoreTest.scala @@ -1,79 +1,79 @@ -/* -Licensed to the Apache Software Foundation (ASF) under one -or more contributor license agreements. See the NOTICE file -distributed with this work for additional information -regarding copyright ownership. The ASF licenses this file -to you under the Apache License, Version 2.0 (the -"License"); you may not use this file except in compliance -with the License. You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, -software distributed under the License is distributed on an -"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -KIND, either express or implied. See the License for the -specific language governing permissions and limitations -under the License. -*/ -package org.apache.griffin.measure.algo.core - -import org.apache.griffin.measure.config.params.user.EvaluateRuleParam -import org.apache.griffin.measure.rule.expr._ -import org.apache.griffin.measure.rule.{RuleAnalyzer, RuleFactory} -import org.junit.runner.RunWith -import org.scalatest.junit.JUnitRunner -import org.scalatest.{BeforeAndAfter, FunSuite, Matchers} -import org.scalatest.PrivateMethodTester - -@RunWith(classOf[JUnitRunner]) -class ProfileCoreTest extends FunSuite with Matchers with BeforeAndAfter with PrivateMethodTester { - - def findExprId(exprs: Iterable[Expr], desc: String): String = { - exprs.find(_.desc == desc) match { - case Some(expr) => expr._id - case _ => "" - } - } - - test ("match data success") { - val rule = "$source.name = 'jack' AND $source.age = null" - val evaluateRuleParam = EvaluateRuleParam(1.0, rule) - val ruleFactory = RuleFactory(evaluateRuleParam) - val statement = ruleFactory.generateRule - val ruleAnalyzer = RuleAnalyzer(statement) - - val sourcePersistExprs = ruleAnalyzer.sourceRuleExprs.persistExprs - - val source = (Map[String, Any]( - (findExprId(sourcePersistExprs, "$source['name']") -> "jack"), - (findExprId(sourcePersistExprs, "$source['age']") -> null) - ), Map[String, Any]()) - - val matchData = PrivateMethod[(Boolean, Map[String, Any])]('matchData) - val result = ProfileCore invokePrivate matchData(source, ruleAnalyzer) - result._1 should be (true) - result._2.size should be (0) - } - - test ("match data fail") { - val rule = "$source.name = 'jack' AND $source.age != null" - val evaluateRuleParam = EvaluateRuleParam(1.0, rule) - val ruleFactory = RuleFactory(evaluateRuleParam) - val statement = ruleFactory.generateRule - val ruleAnalyzer = RuleAnalyzer(statement) - - val sourcePersistExprs = ruleAnalyzer.sourceRuleExprs.persistExprs - - val source = (Map[String, Any]( - (findExprId(sourcePersistExprs, "$source['name']") -> "jack"), - (findExprId(sourcePersistExprs, "$source['age']") -> null) - ), Map[String, Any]()) - - val matchData = PrivateMethod[(Boolean, Map[String, Any])]('matchData) - val result = ProfileCore invokePrivate matchData(source, ruleAnalyzer) - result._1 should be (false) - result._2.size shouldNot be (0) - } - -} +///* +//Licensed to the Apache Software Foundation (ASF) under one +//or more contributor license agreements. See the NOTICE file +//distributed with this work for additional information +//regarding copyright ownership. The ASF licenses this file +//to you under the Apache License, Version 2.0 (the +//"License"); you may not use this file except in compliance +//with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +//Unless required by applicable law or agreed to in writing, +//software distributed under the License is distributed on an +//"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +//KIND, either express or implied. See the License for the +//specific language governing permissions and limitations +//under the License. +//*/ +//package org.apache.griffin.measure.algo.core +// +//import org.apache.griffin.measure.config.params.user.EvaluateRuleParam +//import org.apache.griffin.measure.rule.expr._ +//import org.apache.griffin.measure.rule.{RuleAnalyzer, RuleFactory} +//import org.junit.runner.RunWith +//import org.scalatest.junit.JUnitRunner +//import org.scalatest.{BeforeAndAfter, FunSuite, Matchers} +//import org.scalatest.PrivateMethodTester +// +//@RunWith(classOf[JUnitRunner]) +//class ProfileCoreTest extends FunSuite with Matchers with BeforeAndAfter with PrivateMethodTester { +// +// def findExprId(exprs: Iterable[Expr], desc: String): String = { +// exprs.find(_.desc == desc) match { +// case Some(expr) => expr._id +// case _ => "" +// } +// } +// +// test ("match data success") { +// val rule = "$source.name = 'jack' AND $source.age = null" +// val evaluateRuleParam = EvaluateRuleParam(1.0, rule) +// val ruleFactory = RuleFactory(evaluateRuleParam) +// val statement = ruleFactory.generateRule +// val ruleAnalyzer = RuleAnalyzer(statement) +// +// val sourcePersistExprs = ruleAnalyzer.sourceRuleExprs.persistExprs +// +// val source = (Map[String, Any]( +// (findExprId(sourcePersistExprs, "$source['name']") -> "jack"), +// (findExprId(sourcePersistExprs, "$source['age']") -> null) +// ), Map[String, Any]()) +// +// val matchData = PrivateMethod[(Boolean, Map[String, Any])]('matchData) +// val result = ProfileCore invokePrivate matchData(source, ruleAnalyzer) +// result._1 should be (true) +// result._2.size should be (0) +// } +// +// test ("match data fail") { +// val rule = "$source.name = 'jack' AND $source.age != null" +// val evaluateRuleParam = EvaluateRuleParam(1.0, rule) +// val ruleFactory = RuleFactory(evaluateRuleParam) +// val statement = ruleFactory.generateRule +// val ruleAnalyzer = RuleAnalyzer(statement) +// +// val sourcePersistExprs = ruleAnalyzer.sourceRuleExprs.persistExprs +// +// val source = (Map[String, Any]( +// (findExprId(sourcePersistExprs, "$source['name']") -> "jack"), +// (findExprId(sourcePersistExprs, "$source['age']") -> null) +// ), Map[String, Any]()) +// +// val matchData = PrivateMethod[(Boolean, Map[String, Any])]('matchData) +// val result = ProfileCore invokePrivate matchData(source, ruleAnalyzer) +// result._1 should be (false) +// result._2.size shouldNot be (0) +// } +// +//} diff --git a/measure/src/test/scala/org/apache/griffin/measure/config/reader/ParamRawStringReaderTest.scala b/measure/src/test/scala/org/apache/griffin/measure/config/reader/ParamRawStringReaderTest.scala index b3c94e56f..9e5d380eb 100644 --- a/measure/src/test/scala/org/apache/griffin/measure/config/reader/ParamRawStringReaderTest.scala +++ b/measure/src/test/scala/org/apache/griffin/measure/config/reader/ParamRawStringReaderTest.scala @@ -26,7 +26,7 @@ import org.scalatest.{BeforeAndAfter, FunSuite, Matchers} @RunWith(classOf[JUnitRunner]) class ParamRawStringReaderTest extends FunSuite with Matchers with BeforeAndAfter { - test("read config") { + test("read raw config") { val rawString = """{"type": "hdfs", "config": {"path": "/path/to", "time": 1234567}}""" val reader = ParamRawStringReader(rawString) @@ -34,5 +34,4 @@ class ParamRawStringReaderTest extends FunSuite with Matchers with BeforeAndAfte paramTry.isSuccess should be (true) paramTry.get should be (PersistParam("hdfs", Map[String, Any](("path" -> "/path/to"), ("time" -> 1234567)))) } - } diff --git a/measure/src/test/scala/org/apache/griffin/measure/rule/ExprValueUtilTest.scala b/measure/src/test/scala/org/apache/griffin/measure/rule/ExprValueUtilTest.scala index dd8d4a06c..9cfcab55a 100644 --- a/measure/src/test/scala/org/apache/griffin/measure/rule/ExprValueUtilTest.scala +++ b/measure/src/test/scala/org/apache/griffin/measure/rule/ExprValueUtilTest.scala @@ -1,86 +1,86 @@ -/* -Licensed to the Apache Software Foundation (ASF) under one -or more contributor license agreements. See the NOTICE file -distributed with this work for additional information -regarding copyright ownership. The ASF licenses this file -to you under the Apache License, Version 2.0 (the -"License"); you may not use this file except in compliance -with the License. You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, -software distributed under the License is distributed on an -"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -KIND, either express or implied. See the License for the -specific language governing permissions and limitations -under the License. -*/ -package org.apache.griffin.measure.rule - -import org.apache.griffin.measure.config.params.user.EvaluateRuleParam -import org.apache.griffin.measure.rule.expr.{Expr, StatementExpr} -import org.junit.runner.RunWith -import org.scalatest.junit.JUnitRunner -import org.scalatest.{BeforeAndAfter, FunSuite, Matchers} - -@RunWith(classOf[JUnitRunner]) -class ExprValueUtilTest extends FunSuite with BeforeAndAfter with Matchers { - - test ("rule calculation") { - // val rules = "$source.json().name = 's2' and $source.json().age[*] = 32" - // val rules = "$source.json().items[*] = 202 AND $source.json().age[*] = 32 AND $source.json().df[*].a = 1" - val rules = "$source.json().items[*] = 202 AND $source.json().age[*] = 32 AND $source.json().df['a' = 1].b = 4" - // val rules = "$source.json().df[0].a = 1" - val ep = EvaluateRuleParam(1, rules) - - val ruleFactory = RuleFactory(ep) - val rule: StatementExpr = ruleFactory.generateRule() - val ruleAnalyzer: RuleAnalyzer = RuleAnalyzer(rule) - - val ruleExprs = ruleAnalyzer.sourceRuleExprs - val constFinalExprValueMap = Map[String, Any]() - - val data = List[String]( - ("""{"name": "s1", "age": [22, 23], "items": [102, 104, 106], "df": [{"a": 1, "b": 3}, {"a": 2, "b": 4}]}"""), - ("""{"name": "s2", "age": [32, 33], "items": [202, 204, 206], "df": [{"a": 1, "b": 4}, {"a": 2, "b": 4}]}"""), - ("""{"name": "s3", "age": [42, 43], "items": [302, 304, 306], "df": [{"a": 1, "b": 5}, {"a": 2, "b": 4}]}""") - ) - - def str(expr: Expr) = { - s"${expr._id}: ${expr.desc} [${expr.getClass.getSimpleName}]" - } - println("====") - ruleExprs.finalCacheExprs.foreach { expr => - println(str(expr)) - } - println("====") - ruleExprs.cacheExprs.foreach { expr => - println(str(expr)) - } - - val constExprValueMap = ExprValueUtil.genExprValueMaps(None, ruleAnalyzer.constCacheExprs, Map[String, Any]()) - val finalConstExprValueMap = ExprValueUtil.updateExprValueMaps(ruleAnalyzer.constFinalCacheExprs, constExprValueMap) - val finalConstMap = finalConstExprValueMap.headOption match { - case Some(m) => m - case _ => Map[String, Any]() - } - println("====") - println(ruleAnalyzer.constCacheExprs) - println(ruleAnalyzer.constFinalCacheExprs) - println(finalConstMap) - - println("====") - val valueMaps = data.flatMap { msg => - val cacheExprValueMaps = ExprValueUtil.genExprValueMaps(Some(msg), ruleExprs.cacheExprs, finalConstMap) - val finalExprValueMaps = ExprValueUtil.updateExprValueMaps(ruleExprs.finalCacheExprs, cacheExprValueMaps) - - finalExprValueMaps - } - - valueMaps.foreach(println) - println(valueMaps.size) - - } - -} +///* +//Licensed to the Apache Software Foundation (ASF) under one +//or more contributor license agreements. See the NOTICE file +//distributed with this work for additional information +//regarding copyright ownership. The ASF licenses this file +//to you under the Apache License, Version 2.0 (the +//"License"); you may not use this file except in compliance +//with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +//Unless required by applicable law or agreed to in writing, +//software distributed under the License is distributed on an +//"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +//KIND, either express or implied. See the License for the +//specific language governing permissions and limitations +//under the License. +//*/ +//package org.apache.griffin.measure.rule +// +//import org.apache.griffin.measure.config.params.user.EvaluateRuleParam +//import org.apache.griffin.measure.rule.expr.{Expr, StatementExpr} +//import org.junit.runner.RunWith +//import org.scalatest.junit.JUnitRunner +//import org.scalatest.{BeforeAndAfter, FunSuite, Matchers} +// +//@RunWith(classOf[JUnitRunner]) +//class ExprValueUtilTest extends FunSuite with BeforeAndAfter with Matchers { +// +// test ("rule calculation") { +// // val rules = "$source.json().name = 's2' and $source.json().age[*] = 32" +// // val rules = "$source.json().items[*] = 202 AND $source.json().age[*] = 32 AND $source.json().df[*].a = 1" +// val rules = "$source.json().items[*] = 202 AND $source.json().age[*] = 32 AND $source.json().df['a' = 1].b = 4" +// // val rules = "$source.json().df[0].a = 1" +// val ep = EvaluateRuleParam(1, rules) +// +// val ruleFactory = RuleFactory(ep) +// val rule: StatementExpr = ruleFactory.generateRule() +// val ruleAnalyzer: RuleAnalyzer = RuleAnalyzer(rule) +// +// val ruleExprs = ruleAnalyzer.sourceRuleExprs +// val constFinalExprValueMap = Map[String, Any]() +// +// val data = List[String]( +// ("""{"name": "s1", "age": [22, 23], "items": [102, 104, 106], "df": [{"a": 1, "b": 3}, {"a": 2, "b": 4}]}"""), +// ("""{"name": "s2", "age": [32, 33], "items": [202, 204, 206], "df": [{"a": 1, "b": 4}, {"a": 2, "b": 4}]}"""), +// ("""{"name": "s3", "age": [42, 43], "items": [302, 304, 306], "df": [{"a": 1, "b": 5}, {"a": 2, "b": 4}]}""") +// ) +// +// def str(expr: Expr) = { +// s"${expr._id}: ${expr.desc} [${expr.getClass.getSimpleName}]" +// } +// println("====") +// ruleExprs.finalCacheExprs.foreach { expr => +// println(str(expr)) +// } +// println("====") +// ruleExprs.cacheExprs.foreach { expr => +// println(str(expr)) +// } +// +// val constExprValueMap = ExprValueUtil.genExprValueMaps(None, ruleAnalyzer.constCacheExprs, Map[String, Any]()) +// val finalConstExprValueMap = ExprValueUtil.updateExprValueMaps(ruleAnalyzer.constFinalCacheExprs, constExprValueMap) +// val finalConstMap = finalConstExprValueMap.headOption match { +// case Some(m) => m +// case _ => Map[String, Any]() +// } +// println("====") +// println(ruleAnalyzer.constCacheExprs) +// println(ruleAnalyzer.constFinalCacheExprs) +// println(finalConstMap) +// +// println("====") +// val valueMaps = data.flatMap { msg => +// val cacheExprValueMaps = ExprValueUtil.genExprValueMaps(Some(msg), ruleExprs.cacheExprs, finalConstMap) +// val finalExprValueMaps = ExprValueUtil.updateExprValueMaps(ruleExprs.finalCacheExprs, cacheExprValueMaps) +// +// finalExprValueMaps +// } +// +// valueMaps.foreach(println) +// println(valueMaps.size) +// +// } +// +//} diff --git a/measure/src/test/scala/org/apache/griffin/measure/rule/RuleAnalyzerTest.scala b/measure/src/test/scala/org/apache/griffin/measure/rule/RuleAnalyzerTest.scala index 47b36e805..0b4c16cfa 100644 --- a/measure/src/test/scala/org/apache/griffin/measure/rule/RuleAnalyzerTest.scala +++ b/measure/src/test/scala/org/apache/griffin/measure/rule/RuleAnalyzerTest.scala @@ -1,60 +1,60 @@ -/* -Licensed to the Apache Software Foundation (ASF) under one -or more contributor license agreements. See the NOTICE file -distributed with this work for additional information -regarding copyright ownership. The ASF licenses this file -to you under the Apache License, Version 2.0 (the -"License"); you may not use this file except in compliance -with the License. You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, -software distributed under the License is distributed on an -"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -KIND, either express or implied. See the License for the -specific language governing permissions and limitations -under the License. -*/ -package org.apache.griffin.measure.rule - -import org.apache.griffin.measure.config.params.user.EvaluateRuleParam -import org.junit.runner.RunWith -import org.scalatest.junit.JUnitRunner -import org.scalatest.{BeforeAndAfter, FunSuite, Matchers} - -@RunWith(classOf[JUnitRunner]) -class RuleAnalyzerTest extends FunSuite with BeforeAndAfter with Matchers { - - test ("rule analyze") { - val rule = "$source.name = $target.name AND $source.age = $target.age + (2 * 5) AND $source.born > (6 - 2 * 2)" - val evaluateRuleParam = EvaluateRuleParam(1.0, rule) - val ruleFactory = RuleFactory(evaluateRuleParam) - val statement = ruleFactory.generateRule - - val ruleAnalyzer = RuleAnalyzer(statement) - - ruleAnalyzer.constCacheExprs.map(_.desc) should be (List[String]("2 * 5", "2 * 2", "6 - 2 * 2")) - ruleAnalyzer.constFinalCacheExprs.map(_.desc) should be (Set[String]("2 * 5", "6 - 2 * 2")) - - ruleAnalyzer.sourceRuleExprs.groupbyExprs.map(_.desc) should be (List[String]( - "$source['name']", "$source['age']")) - ruleAnalyzer.sourceRuleExprs.cacheExprs.map(_.desc) should be (List[String]( - "$source['name']", "$source['age']", "$source['born']", "$source['born'] > 6 - 2 * 2")) - ruleAnalyzer.sourceRuleExprs.finalCacheExprs.map(_.desc) should be (Set[String]( - "$source['name']", "$source['age']", "$source['born']", "$source['born'] > 6 - 2 * 2")) - ruleAnalyzer.sourceRuleExprs.persistExprs.map(_.desc) should be (List[String]( - "$source['name']", "$source['age']", "$source['born']")) - - ruleAnalyzer.targetRuleExprs.groupbyExprs.map(_.desc) should be (List[String]( - "$target['name']", "$target['age'] + 2 * 5")) - ruleAnalyzer.targetRuleExprs.cacheExprs.map(_.desc) should be (List[String]( - "$target['name']", "$target['age']", "$target['age'] + 2 * 5")) - ruleAnalyzer.targetRuleExprs.finalCacheExprs.map(_.desc) should be (Set[String]( - "$target['name']", "$target['age']", "$target['age'] + 2 * 5")) - ruleAnalyzer.targetRuleExprs.persistExprs.map(_.desc) should be (List[String]( - "$target['name']", "$target['age']")) - - } - -} +///* +//Licensed to the Apache Software Foundation (ASF) under one +//or more contributor license agreements. See the NOTICE file +//distributed with this work for additional information +//regarding copyright ownership. The ASF licenses this file +//to you under the Apache License, Version 2.0 (the +//"License"); you may not use this file except in compliance +//with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +//Unless required by applicable law or agreed to in writing, +//software distributed under the License is distributed on an +//"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +//KIND, either express or implied. See the License for the +//specific language governing permissions and limitations +//under the License. +//*/ +//package org.apache.griffin.measure.rule +// +//import org.apache.griffin.measure.config.params.user.EvaluateRuleParam +//import org.junit.runner.RunWith +//import org.scalatest.junit.JUnitRunner +//import org.scalatest.{BeforeAndAfter, FunSuite, Matchers} +// +//@RunWith(classOf[JUnitRunner]) +//class RuleAnalyzerTest extends FunSuite with BeforeAndAfter with Matchers { +// +// test ("rule analyze") { +// val rule = "$source.name = $target.name AND $source.age = $target.age + (2 * 5) AND $source.born > (6 - 2 * 2)" +// val evaluateRuleParam = EvaluateRuleParam(1.0, rule) +// val ruleFactory = RuleFactory(evaluateRuleParam) +// val statement = ruleFactory.generateRule +// +// val ruleAnalyzer = RuleAnalyzer(statement) +// +// ruleAnalyzer.constCacheExprs.map(_.desc) should be (List[String]("2 * 5", "2 * 2", "6 - 2 * 2")) +// ruleAnalyzer.constFinalCacheExprs.map(_.desc) should be (Set[String]("2 * 5", "6 - 2 * 2")) +// +// ruleAnalyzer.sourceRuleExprs.groupbyExprs.map(_.desc) should be (List[String]( +// "$source['name']", "$source['age']")) +// ruleAnalyzer.sourceRuleExprs.cacheExprs.map(_.desc) should be (List[String]( +// "$source['name']", "$source['age']", "$source['born']", "$source['born'] > 6 - 2 * 2")) +// ruleAnalyzer.sourceRuleExprs.finalCacheExprs.map(_.desc) should be (Set[String]( +// "$source['name']", "$source['age']", "$source['born']", "$source['born'] > 6 - 2 * 2")) +// ruleAnalyzer.sourceRuleExprs.persistExprs.map(_.desc) should be (List[String]( +// "$source['name']", "$source['age']", "$source['born']")) +// +// ruleAnalyzer.targetRuleExprs.groupbyExprs.map(_.desc) should be (List[String]( +// "$target['name']", "$target['age'] + 2 * 5")) +// ruleAnalyzer.targetRuleExprs.cacheExprs.map(_.desc) should be (List[String]( +// "$target['name']", "$target['age']", "$target['age'] + 2 * 5")) +// ruleAnalyzer.targetRuleExprs.finalCacheExprs.map(_.desc) should be (Set[String]( +// "$target['name']", "$target['age']", "$target['age'] + 2 * 5")) +// ruleAnalyzer.targetRuleExprs.persistExprs.map(_.desc) should be (List[String]( +// "$target['name']", "$target['age']")) +// +// } +// +//} diff --git a/measure/src/test/scala/org/apache/griffin/measure/rule/RuleFactoryTest.scala b/measure/src/test/scala/org/apache/griffin/measure/rule/RuleFactoryTest.scala index c14cd0456..9ed641a0c 100644 --- a/measure/src/test/scala/org/apache/griffin/measure/rule/RuleFactoryTest.scala +++ b/measure/src/test/scala/org/apache/griffin/measure/rule/RuleFactoryTest.scala @@ -1,44 +1,44 @@ -/* -Licensed to the Apache Software Foundation (ASF) under one -or more contributor license agreements. See the NOTICE file -distributed with this work for additional information -regarding copyright ownership. The ASF licenses this file -to you under the Apache License, Version 2.0 (the -"License"); you may not use this file except in compliance -with the License. You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, -software distributed under the License is distributed on an -"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -KIND, either express or implied. See the License for the -specific language governing permissions and limitations -under the License. -*/ -package org.apache.griffin.measure.rule - -import org.apache.griffin.measure.config.params.user.EvaluateRuleParam -import org.junit.runner.RunWith -import org.scalatest.junit.JUnitRunner -import org.scalatest.{BeforeAndAfter, FunSuite, Matchers} - -@RunWith(classOf[JUnitRunner]) -class RuleFactoryTest extends FunSuite with BeforeAndAfter with Matchers { - - test ("generate rule") { - val rule = "$source.name = $target.name AND $source.age = $target.age" - val evaluateRuleParam = EvaluateRuleParam(1.0, rule) - val ruleFactory = RuleFactory(evaluateRuleParam) - ruleFactory.generateRule.desc should be ("$source['name'] = $target['name'] AND $source['age'] = $target['age']") - - val wrong_rule = "$source.name = $target.name AND $source.age = $target1.age" - val evaluateRuleParam1 = EvaluateRuleParam(1.0, wrong_rule) - val ruleFactory1 = RuleFactory(evaluateRuleParam1) - val thrown = intercept[Exception] { - ruleFactory1.generateRule - } - thrown.getMessage should be ("parse rule error!") - } - -} +///* +//Licensed to the Apache Software Foundation (ASF) under one +//or more contributor license agreements. See the NOTICE file +//distributed with this work for additional information +//regarding copyright ownership. The ASF licenses this file +//to you under the Apache License, Version 2.0 (the +//"License"); you may not use this file except in compliance +//with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +//Unless required by applicable law or agreed to in writing, +//software distributed under the License is distributed on an +//"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +//KIND, either express or implied. See the License for the +//specific language governing permissions and limitations +//under the License. +//*/ +//package org.apache.griffin.measure.rule +// +//import org.apache.griffin.measure.config.params.user.EvaluateRuleParam +//import org.junit.runner.RunWith +//import org.scalatest.junit.JUnitRunner +//import org.scalatest.{BeforeAndAfter, FunSuite, Matchers} +// +//@RunWith(classOf[JUnitRunner]) +//class RuleFactoryTest extends FunSuite with BeforeAndAfter with Matchers { +// +// test ("generate rule") { +// val rule = "$source.name = $target.name AND $source.age = $target.age" +// val evaluateRuleParam = EvaluateRuleParam(1.0, rule) +// val ruleFactory = RuleFactory(evaluateRuleParam) +// ruleFactory.generateRule.desc should be ("$source['name'] = $target['name'] AND $source['age'] = $target['age']") +// +// val wrong_rule = "$source.name = $target.name AND $source.age = $target1.age" +// val evaluateRuleParam1 = EvaluateRuleParam(1.0, wrong_rule) +// val ruleFactory1 = RuleFactory(evaluateRuleParam1) +// val thrown = intercept[Exception] { +// ruleFactory1.generateRule +// } +// thrown.getMessage should be ("parse rule error!") +// } +// +//} diff --git a/measure/src/test/scala/org/apache/griffin/measure/rule/RuleParserTest.scala b/measure/src/test/scala/org/apache/griffin/measure/rule/RuleParserTest.scala index 1d153757d..15f1a564c 100644 --- a/measure/src/test/scala/org/apache/griffin/measure/rule/RuleParserTest.scala +++ b/measure/src/test/scala/org/apache/griffin/measure/rule/RuleParserTest.scala @@ -1,213 +1,213 @@ -/* -Licensed to the Apache Software Foundation (ASF) under one -or more contributor license agreements. See the NOTICE file -distributed with this work for additional information -regarding copyright ownership. The ASF licenses this file -to you under the Apache License, Version 2.0 (the -"License"); you may not use this file except in compliance -with the License. You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, -software distributed under the License is distributed on an -"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -KIND, either express or implied. See the License for the -specific language governing permissions and limitations -under the License. -*/ -package org.apache.griffin.measure.rule - -import org.junit.runner.RunWith -import org.scalatest.junit.JUnitRunner -import org.scalatest.{BeforeAndAfter, FunSuite, Matchers} -//import org.scalatest.FlatSpec -//import org.scalamock.scalatest.MockFactory - -@RunWith(classOf[JUnitRunner]) -class RuleParserTest extends FunSuite with Matchers with BeforeAndAfter { - - val ruleParser = RuleParser() - - test ("literal number") { - val rule1 = "123" - val result1 = ruleParser.parseAll(ruleParser.literal, rule1) - result1.successful should be (true) - result1.get.value should be (Some(123)) - - val rule2 = "12.3" - val result2 = ruleParser.parseAll(ruleParser.literal, rule2) - result2.successful should be (true) - result2.get.value should be (Some(12.3)) - } - - test ("literial string") { - val rule1 = "'123'" - val result1 = ruleParser.parseAll(ruleParser.literal, rule1) - result1.successful should be (true) - result1.get.value should be (Some("123")) - - val rule2 = "\"123\"" - val result2 = ruleParser.parseAll(ruleParser.literal, rule1) - result2.successful should be (true) - result2.get.value should be (Some("123")) - - val rule3 = "'1+2-3'" - val result3 = ruleParser.parseAll(ruleParser.literal, rule3) - result3.successful should be (true) - result3.get.value should be (Some("1+2-3")) - } - - test ("literial time") { - val rule = "3h" - val result = ruleParser.parseAll(ruleParser.literal, rule) - result.successful should be (true) - result.get.value should be (Some(3*3600*1000)) - } - - test ("literial boolean") { - val rule = "true" - val result = ruleParser.parseAll(ruleParser.literal, rule) - result.successful should be (true) - result.get.value should be (Some(true)) - } - - test ("literial null") { - val rule = "null" - val result = ruleParser.parseAll(ruleParser.literal, rule) - result.successful should be (true) - result.get.value should be (Some(null)) - } - - test ("literial none") { - val rule = "none" - val result = ruleParser.parseAll(ruleParser.literal, rule) - result.successful should be (true) - result.get.value should be (None) - } - - test ("selection head") { - val rule = "$source" - val result = ruleParser.parseAll(ruleParser.selectionHead, rule) - result.successful should be (true) - result.get.head should be ("source") - } - - test ("field select") { - val rule = ".name" - val result = ruleParser.parseAll(ruleParser.selector, rule) - result.successful should be (true) - result.get.desc should be ("['name']") - } - - test ("function operation") { - val rule = ".func(1, 'abc', 3 + 4)" - val result = ruleParser.parseAll(ruleParser.selector, rule) - result.successful should be (true) - result.get.desc should be (".func(1, 'abc', 3 + 4)") - } - - test ("index field range select") { - val rule1 = "['field']" - val result1 = ruleParser.parseAll(ruleParser.selector, rule1) - result1.successful should be (true) - result1.get.desc should be ("['field']") - - val rule2 = "[1, 4]" - val result2 = ruleParser.parseAll(ruleParser.selector, rule2) - result2.successful should be (true) - result2.get.desc should be ("[1, 4]") - - val rule3 = "[1, 'name', 'age', 5, (6, 8)]" - val result3 = ruleParser.parseAll(ruleParser.selector, rule3) - result3.successful should be (true) - result3.get.desc should be ("[1, 'name', 'age', 5, (6, 8)]") - } - - test ("index field range") { - val rule1 = "(3, 5)" - val result1 = ruleParser.parseAll(ruleParser.indexFieldRange, rule1) - result1.successful should be (true) - result1.get.desc should be ("(3, 5)") - - val rule2 = "'name'" - val result2 = ruleParser.parseAll(ruleParser.indexFieldRange, rule2) - result2.successful should be (true) - result2.get.desc should be ("'name'") - - val rule3 = "*" - val result3 = ruleParser.parseAll(ruleParser.indexFieldRange, rule3) - result3.successful should be (true) - result3.get.desc should be ("*") - } - - test ("filter select") { - val rule = "['age' > 16]" - val result = ruleParser.parseAll(ruleParser.selector, rule) - result.successful should be (true) - result.get.desc should be ("['age' > 16]") - } - - test ("selection") { - val rule = "$source['age' > 16].func(1, 'abc')[1, 3, 'name'].time[*]" - val result = ruleParser.parseAll(ruleParser.selection, rule) - result.successful should be (true) - result.get.desc should be ("$source['age' > 16].func(1, 'abc')[1, 3, 'name']['time'][*]") - } - - test ("math expr") { - val rule = "$source.age * 6 + 4 / 2" - val result = ruleParser.parseAll(ruleParser.mathExpr, rule) - result.successful should be (true) - result.get.desc should be ("$source['age'] * 6 + 4 / 2") - - val rule2 = "'age + 1' / 'vv'" - val result2 = ruleParser.parseAll(ruleParser.mathExpr, rule2) - result2.successful should be (true) - result2.get.desc should be ("'age + 1' / 'vv'") - println(result2) - } - - test ("range expr") { - val rule = "($source.age + 1, $target.age + 3, 40)" - val result = ruleParser.parseAll(ruleParser.rangeExpr, rule) - result.successful should be (true) - result.get.desc should be ("($source['age'] + 1, $target['age'] + 3, 40)") - } - - test ("logical expr") { - val rule1 = "$source.age + 1 = $target.age" - val result1 = ruleParser.parseAll(ruleParser.logicalExpr, rule1) - result1.successful should be (true) - result1.get.desc should be ("$source['age'] + 1 = $target['age']") - - val rule2 = "$source.age in (3, 5, 6, 10)" - val result2 = ruleParser.parseAll(ruleParser.logicalExpr, rule2) - result2.successful should be (true) - result2.get.desc should be ("$source['age'] in (3, 5, 6, 10)") - } - - test ("logical statement") { - val rule1 = "$source.descs[0] = $target.desc AND $source.name = $target.name" - val result1 = ruleParser.parseAll(ruleParser.logicalStatement, rule1) - result1.successful should be (true) - result1.get.desc should be ("$source['descs'][0] = $target['desc'] AND $source['name'] = $target['name']") - - val rule2 = "NOT $source.age = $target.age" - val result2 = ruleParser.parseAll(ruleParser.logicalStatement, rule2) - result2.successful should be (true) - result2.get.desc should be ("NOT $source['age'] = $target['age']") - } - - test ("whole rule") { - val rule1 = "$source.name = $target.name AND $source.age = $target.age" - val result1 = ruleParser.parseAll(ruleParser.rule, rule1) - result1.successful should be (true) - result1.get.desc should be ("$source['name'] = $target['name'] AND $source['age'] = $target['age']") - - val rule2 = "$source.name = $target.name AND $source.age = $target.age AND $source.id > 1000" - val result2 = ruleParser.parseAll(ruleParser.rule, rule2) - result2.successful should be (true) - result2.get.desc should be ("$source['name'] = $target['name'] AND $source['age'] = $target['age'] AND $source['id'] > 1000") - } -} +///* +//Licensed to the Apache Software Foundation (ASF) under one +//or more contributor license agreements. See the NOTICE file +//distributed with this work for additional information +//regarding copyright ownership. The ASF licenses this file +//to you under the Apache License, Version 2.0 (the +//"License"); you may not use this file except in compliance +//with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +//Unless required by applicable law or agreed to in writing, +//software distributed under the License is distributed on an +//"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +//KIND, either express or implied. See the License for the +//specific language governing permissions and limitations +//under the License. +//*/ +//package org.apache.griffin.measure.rule +// +//import org.junit.runner.RunWith +//import org.scalatest.junit.JUnitRunner +//import org.scalatest.{BeforeAndAfter, FunSuite, Matchers} +////import org.scalatest.FlatSpec +////import org.scalamock.scalatest.MockFactory +// +//@RunWith(classOf[JUnitRunner]) +//class RuleParserTest extends FunSuite with Matchers with BeforeAndAfter { +// +// val ruleParser = RuleParser() +// +// test ("literal number") { +// val rule1 = "123" +// val result1 = ruleParser.parseAll(ruleParser.literal, rule1) +// result1.successful should be (true) +// result1.get.value should be (Some(123)) +// +// val rule2 = "12.3" +// val result2 = ruleParser.parseAll(ruleParser.literal, rule2) +// result2.successful should be (true) +// result2.get.value should be (Some(12.3)) +// } +// +// test ("literial string") { +// val rule1 = "'123'" +// val result1 = ruleParser.parseAll(ruleParser.literal, rule1) +// result1.successful should be (true) +// result1.get.value should be (Some("123")) +// +// val rule2 = "\"123\"" +// val result2 = ruleParser.parseAll(ruleParser.literal, rule1) +// result2.successful should be (true) +// result2.get.value should be (Some("123")) +// +// val rule3 = "'1+2-3'" +// val result3 = ruleParser.parseAll(ruleParser.literal, rule3) +// result3.successful should be (true) +// result3.get.value should be (Some("1+2-3")) +// } +// +// test ("literial time") { +// val rule = "3h" +// val result = ruleParser.parseAll(ruleParser.literal, rule) +// result.successful should be (true) +// result.get.value should be (Some(3*3600*1000)) +// } +// +// test ("literial boolean") { +// val rule = "true" +// val result = ruleParser.parseAll(ruleParser.literal, rule) +// result.successful should be (true) +// result.get.value should be (Some(true)) +// } +// +// test ("literial null") { +// val rule = "null" +// val result = ruleParser.parseAll(ruleParser.literal, rule) +// result.successful should be (true) +// result.get.value should be (Some(null)) +// } +// +// test ("literial none") { +// val rule = "none" +// val result = ruleParser.parseAll(ruleParser.literal, rule) +// result.successful should be (true) +// result.get.value should be (None) +// } +// +// test ("selection head") { +// val rule = "$source" +// val result = ruleParser.parseAll(ruleParser.selectionHead, rule) +// result.successful should be (true) +// result.get.head should be ("source") +// } +// +// test ("field select") { +// val rule = ".name" +// val result = ruleParser.parseAll(ruleParser.selector, rule) +// result.successful should be (true) +// result.get.desc should be ("['name']") +// } +// +// test ("function operation") { +// val rule = ".func(1, 'abc', 3 + 4)" +// val result = ruleParser.parseAll(ruleParser.selector, rule) +// result.successful should be (true) +// result.get.desc should be (".func(1, 'abc', 3 + 4)") +// } +// +// test ("index field range select") { +// val rule1 = "['field']" +// val result1 = ruleParser.parseAll(ruleParser.selector, rule1) +// result1.successful should be (true) +// result1.get.desc should be ("['field']") +// +// val rule2 = "[1, 4]" +// val result2 = ruleParser.parseAll(ruleParser.selector, rule2) +// result2.successful should be (true) +// result2.get.desc should be ("[1, 4]") +// +// val rule3 = "[1, 'name', 'age', 5, (6, 8)]" +// val result3 = ruleParser.parseAll(ruleParser.selector, rule3) +// result3.successful should be (true) +// result3.get.desc should be ("[1, 'name', 'age', 5, (6, 8)]") +// } +// +// test ("index field range") { +// val rule1 = "(3, 5)" +// val result1 = ruleParser.parseAll(ruleParser.indexFieldRange, rule1) +// result1.successful should be (true) +// result1.get.desc should be ("(3, 5)") +// +// val rule2 = "'name'" +// val result2 = ruleParser.parseAll(ruleParser.indexFieldRange, rule2) +// result2.successful should be (true) +// result2.get.desc should be ("'name'") +// +// val rule3 = "*" +// val result3 = ruleParser.parseAll(ruleParser.indexFieldRange, rule3) +// result3.successful should be (true) +// result3.get.desc should be ("*") +// } +// +// test ("filter select") { +// val rule = "['age' > 16]" +// val result = ruleParser.parseAll(ruleParser.selector, rule) +// result.successful should be (true) +// result.get.desc should be ("['age' > 16]") +// } +// +// test ("selection") { +// val rule = "$source['age' > 16].func(1, 'abc')[1, 3, 'name'].time[*]" +// val result = ruleParser.parseAll(ruleParser.selection, rule) +// result.successful should be (true) +// result.get.desc should be ("$source['age' > 16].func(1, 'abc')[1, 3, 'name']['time'][*]") +// } +// +// test ("math expr") { +// val rule = "$source.age * 6 + 4 / 2" +// val result = ruleParser.parseAll(ruleParser.mathExpr, rule) +// result.successful should be (true) +// result.get.desc should be ("$source['age'] * 6 + 4 / 2") +// +// val rule2 = "'age + 1' / 'vv'" +// val result2 = ruleParser.parseAll(ruleParser.mathExpr, rule2) +// result2.successful should be (true) +// result2.get.desc should be ("'age + 1' / 'vv'") +// println(result2) +// } +// +// test ("range expr") { +// val rule = "($source.age + 1, $target.age + 3, 40)" +// val result = ruleParser.parseAll(ruleParser.rangeExpr, rule) +// result.successful should be (true) +// result.get.desc should be ("($source['age'] + 1, $target['age'] + 3, 40)") +// } +// +// test ("logical expr") { +// val rule1 = "$source.age + 1 = $target.age" +// val result1 = ruleParser.parseAll(ruleParser.logicalExpr, rule1) +// result1.successful should be (true) +// result1.get.desc should be ("$source['age'] + 1 = $target['age']") +// +// val rule2 = "$source.age in (3, 5, 6, 10)" +// val result2 = ruleParser.parseAll(ruleParser.logicalExpr, rule2) +// result2.successful should be (true) +// result2.get.desc should be ("$source['age'] in (3, 5, 6, 10)") +// } +// +// test ("logical statement") { +// val rule1 = "$source.descs[0] = $target.desc AND $source.name = $target.name" +// val result1 = ruleParser.parseAll(ruleParser.logicalStatement, rule1) +// result1.successful should be (true) +// result1.get.desc should be ("$source['descs'][0] = $target['desc'] AND $source['name'] = $target['name']") +// +// val rule2 = "NOT $source.age = $target.age" +// val result2 = ruleParser.parseAll(ruleParser.logicalStatement, rule2) +// result2.successful should be (true) +// result2.get.desc should be ("NOT $source['age'] = $target['age']") +// } +// +// test ("whole rule") { +// val rule1 = "$source.name = $target.name AND $source.age = $target.age" +// val result1 = ruleParser.parseAll(ruleParser.rule, rule1) +// result1.successful should be (true) +// result1.get.desc should be ("$source['name'] = $target['name'] AND $source['age'] = $target['age']") +// +// val rule2 = "$source.name = $target.name AND $source.age = $target.age AND $source.id > 1000" +// val result2 = ruleParser.parseAll(ruleParser.rule, rule2) +// result2.successful should be (true) +// result2.get.desc should be ("$source['name'] = $target['name'] AND $source['age'] = $target['age'] AND $source['id'] > 1000") +// } +//} diff --git a/measure/src/test/scala/org/apache/griffin/measure/sql/SqlTest.scala b/measure/src/test/scala/org/apache/griffin/measure/sql/SqlTest.scala index 1e05198a9..7b2306259 100644 --- a/measure/src/test/scala/org/apache/griffin/measure/sql/SqlTest.scala +++ b/measure/src/test/scala/org/apache/griffin/measure/sql/SqlTest.scala @@ -1,125 +1,125 @@ -package org.apache.griffin.measure.sql - -import org.apache.griffin.measure.config.params.user.EvaluateRuleParam -import org.apache.griffin.measure.rule.expr.{Expr, StatementExpr} -import org.apache.spark.sql.{DataFrame, SQLContext} -import org.apache.spark.sql.types.{ArrayType, IntegerType, StructField, StructType} -import org.apache.spark.{SparkConf, SparkContext} -import org.junit.runner.RunWith -import org.scalatest.junit.JUnitRunner -import org.scalatest.{BeforeAndAfter, FunSuite, Matchers} - -@RunWith(classOf[JUnitRunner]) -class SqlTest extends FunSuite with BeforeAndAfter with Matchers { - - var sc: SparkContext = _ - var sqlContext: SQLContext = _ - - before { - val conf = new SparkConf().setMaster("local[*]").setAppName("test") - sc = new SparkContext(conf) - sqlContext = new SQLContext(sc) - } - - test ("spark sql") { - - val squared = (s: Int) => { - s * s - } - sqlContext.udf.register("square", squared) - - val a = sqlContext.range(1, 20) - a.show - - a.registerTempTable("test") - - val table = sqlContext.sql("select * from test") - table.show() - - val result = sqlContext.sql("select id, square(id) as id_squared from test") - result.show() - - } - - test ("json") { - def jsonToDataFrame(json: String, schema: Option[StructType] = None): DataFrame = { - val reader = sqlContext.read - val rd = schema match { - case Some(scm) => reader.schema(scm) - case _ => reader - } - rd.json(sc.parallelize(json :: Nil)) - } - - val json = - """ - |{ - | "a": [ - | 1, 2, 3 - | ] - |} - """.stripMargin - -// val bt = StructField("b", IntegerType) -// val at = StructField("a", StructType(bt :: Nil)) +//package org.apache.griffin.measure.sql +// +//import org.apache.griffin.measure.config.params.user.EvaluateRuleParam +//import org.apache.griffin.measure.rule.expr.{Expr, StatementExpr} +//import org.apache.spark.sql.{DataFrame, SQLContext} +//import org.apache.spark.sql.types.{ArrayType, IntegerType, StructField, StructType} +//import org.apache.spark.{SparkConf, SparkContext} +//import org.junit.runner.RunWith +//import org.scalatest.junit.JUnitRunner +//import org.scalatest.{BeforeAndAfter, FunSuite, Matchers} +// +//@RunWith(classOf[JUnitRunner]) +//class SqlTest extends FunSuite with BeforeAndAfter with Matchers { +// +// var sc: SparkContext = _ +// var sqlContext: SQLContext = _ +// +// before { +// val conf = new SparkConf().setMaster("local[*]").setAppName("test") +// sc = new SparkContext(conf) +// sqlContext = new SQLContext(sc) +// } +// +// test ("spark sql") { +// +// val squared = (s: Int) => { +// s * s +// } +// sqlContext.udf.register("square", squared) +// +// val a = sqlContext.range(1, 20) +// a.show +// +// a.registerTempTable("test") +// +// val table = sqlContext.sql("select * from test") +// table.show() +// +// val result = sqlContext.sql("select id, square(id) as id_squared from test") +// result.show() +// +// } +// +// test ("json") { +// def jsonToDataFrame(json: String, schema: Option[StructType] = None): DataFrame = { +// val reader = sqlContext.read +// val rd = schema match { +// case Some(scm) => reader.schema(scm) +// case _ => reader +// } +// rd.json(sc.parallelize(json :: Nil)) +// } +// +// val json = +// """ +// |{ +// | "a": [ +// | 1, 2, 3 +// | ] +// |} +// """.stripMargin +// +//// val bt = StructField("b", IntegerType) +//// val at = StructField("a", StructType(bt :: Nil)) +//// val schema = StructType(at :: Nil) +// +// val at = StructField("a", ArrayType(IntegerType)) // val schema = StructType(at :: Nil) - - val at = StructField("a", ArrayType(IntegerType)) - val schema = StructType(at :: Nil) - - val df = jsonToDataFrame(json, Some(schema)) - - df.registerTempTable("json") - - val result = sqlContext.sql("select a[1] from json") - result.show - - } - - test ("json file") { - - // read json file directly +// +// val df = jsonToDataFrame(json, Some(schema)) +// +// df.registerTempTable("json") +// +// val result = sqlContext.sql("select a[1] from json") +// result.show +// +// } +// +// test ("json file") { +// +// // read json file directly +//// val filePath = "src/test/resources/test-data.jsonFile" +//// val reader = sqlContext.read +//// val df = reader.json(filePath) +//// df.show +//// +//// df.registerTempTable("ttt") +//// val result = sqlContext.sql("select * from ttt where list[0].c = 11") +//// result.show +// +// // whole json file +//// val filePath = "src/test/resources/test-data0.json" +////// val filePath = "hdfs://localhost/test/file/t1.json" +//// val jsonRDD = sc.wholeTextFiles(s"${filePath},${filePath}").map(x => x._2) +//// val namesJson = sqlContext.read.json(jsonRDD) +//// namesJson.printSchema +//// namesJson.show +// +// // read text file then convert to json // val filePath = "src/test/resources/test-data.jsonFile" +// val rdd = sc.textFile(filePath) // val reader = sqlContext.read -// val df = reader.json(filePath) +// val df = reader.json(rdd) // df.show +// df.printSchema // // df.registerTempTable("ttt") // val result = sqlContext.sql("select * from ttt where list[0].c = 11") // result.show - - // whole json file -// val filePath = "src/test/resources/test-data0.json" -//// val filePath = "hdfs://localhost/test/file/t1.json" -// val jsonRDD = sc.wholeTextFiles(s"${filePath},${filePath}").map(x => x._2) -// val namesJson = sqlContext.read.json(jsonRDD) -// namesJson.printSchema -// namesJson.show - - // read text file then convert to json - val filePath = "src/test/resources/test-data.jsonFile" - val rdd = sc.textFile(filePath) - val reader = sqlContext.read - val df = reader.json(rdd) - df.show - df.printSchema - - df.registerTempTable("ttt") - val result = sqlContext.sql("select * from ttt where list[0].c = 11") - result.show - - // udf - val slice = (arr: Seq[Long], f: Int, e: Int) => arr.slice(f, e) -// val slice = (arr: Seq[Long]) => arr.slice(0, 1) - sqlContext.udf.register("slice", slice) - - val result1 = sqlContext.sql("select slice(t, 0, 2) from ttt") - result1.show - - } - - test ("accu sql") { -// val file1 = - } - -} +// +// // udf +// val slice = (arr: Seq[Long], f: Int, e: Int) => arr.slice(f, e) +//// val slice = (arr: Seq[Long]) => arr.slice(0, 1) +// sqlContext.udf.register("slice", slice) +// +// val result1 = sqlContext.sql("select slice(t, 0, 2) from ttt") +// result1.show +// +// } +// +// test ("accu sql") { +//// val file1 = +// } +// +//} diff --git a/measure/src/test/scala/org/apache/griffin/measure/utils/JsonUtilTest.scala b/measure/src/test/scala/org/apache/griffin/measure/utils/JsonUtilTest.scala index 4daebb6b7..233d78cb5 100644 --- a/measure/src/test/scala/org/apache/griffin/measure/utils/JsonUtilTest.scala +++ b/measure/src/test/scala/org/apache/griffin/measure/utils/JsonUtilTest.scala @@ -1,60 +1,60 @@ -/* -Licensed to the Apache Software Foundation (ASF) under one -or more contributor license agreements. See the NOTICE file -distributed with this work for additional information -regarding copyright ownership. The ASF licenses this file -to you under the Apache License, Version 2.0 (the -"License"); you may not use this file except in compliance -with the License. You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, -software distributed under the License is distributed on an -"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -KIND, either express or implied. See the License for the -specific language governing permissions and limitations -under the License. -*/ -package org.apache.griffin.measure.utils - -import org.junit.runner.RunWith -import org.scalatest.junit.JUnitRunner -import org.scalatest.{BeforeAndAfter, FunSuite, Matchers} - - -@RunWith(classOf[JUnitRunner]) -class JsonUtilTest extends FunSuite with Matchers with BeforeAndAfter { - - val map = Map[String, Any](("name" -> "test"), ("age" -> 15)) - val json = """{"name":"test","age":15}""" - - val person = JsonUtilTest.Person("test", 15) - - test ("toJson 1") { - val symbolMap = map.map(p => (Symbol(p._1), p._2)) - JsonUtil.toJson(symbolMap) should equal (json) - } - - test ("toJson 2") { - JsonUtil.toJson(map) should equal (json) - } - - test ("toMap") { - JsonUtil.toMap(json) should equal (map) - } - - test ("fromJson 1") { - JsonUtil.fromJson[JsonUtilTest.Person](json) should equal (person) - } - - test ("fromJson 2") { - val is = new java.io.ByteArrayInputStream(json.getBytes("utf-8")); - JsonUtil.fromJson[JsonUtilTest.Person](is) should equal (person) - } - -} - -object JsonUtilTest { - case class Person(name: String, age: Int){} -} +///* +//Licensed to the Apache Software Foundation (ASF) under one +//or more contributor license agreements. See the NOTICE file +//distributed with this work for additional information +//regarding copyright ownership. The ASF licenses this file +//to you under the Apache License, Version 2.0 (the +//"License"); you may not use this file except in compliance +//with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +//Unless required by applicable law or agreed to in writing, +//software distributed under the License is distributed on an +//"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +//KIND, either express or implied. See the License for the +//specific language governing permissions and limitations +//under the License. +//*/ +//package org.apache.griffin.measure.utils +// +//import org.junit.runner.RunWith +//import org.scalatest.junit.JUnitRunner +//import org.scalatest.{BeforeAndAfter, FunSuite, Matchers} +// +// +//@RunWith(classOf[JUnitRunner]) +//class JsonUtilTest extends FunSuite with Matchers with BeforeAndAfter { +// +// val map = Map[String, Any](("name" -> "test"), ("age" -> 15)) +// val json = """{"name":"test","age":15}""" +// +// val person = JsonUtilTest.Person("test", 15) +// +// test ("toJson 1") { +// val symbolMap = map.map(p => (Symbol(p._1), p._2)) +// JsonUtil.toJson(symbolMap) should equal (json) +// } +// +// test ("toJson 2") { +// JsonUtil.toJson(map) should equal (json) +// } +// +// test ("toMap") { +// JsonUtil.toMap(json) should equal (map) +// } +// +// test ("fromJson 1") { +// JsonUtil.fromJson[JsonUtilTest.Person](json) should equal (person) +// } +// +// test ("fromJson 2") { +// val is = new java.io.ByteArrayInputStream(json.getBytes("utf-8")); +// JsonUtil.fromJson[JsonUtilTest.Person](is) should equal (person) +// } +// +//} +// +//object JsonUtilTest { +// case class Person(name: String, age: Int){} +//} From 0c615b62dfd89001cad99695a988fe67428509b6 Mon Sep 17 00:00:00 2001 From: Lionel Liu Date: Fri, 25 Aug 2017 16:39:57 +0800 Subject: [PATCH 051/111] not done --- measure/src/main/resources/config.json | 21 +- .../apache/griffin/measure/Application.scala | 255 ++++++---- .../algo/batch/BatchAccuracyAlgo.scala | 4 +- .../measure/algo/batch/BatchProfileAlgo.scala | 4 +- .../streaming/StreamingAccuracyAlgo.scala | 4 +- .../streaming/StreamingAccuracyProcess.scala | 468 ++++++++--------- .../params/user/EvaluateRuleParam.scala | 2 +- .../config/params/user/RuleParam.scala | 10 +- .../connector/DataConnectorFactory.scala | 139 ----- .../connector/cache/DataCacheable.scala | 86 ---- .../cache/DfCacheDataConnector.scala | 281 ---------- .../cache/HiveCacheDataConnector.scala | 351 ------------- .../cache/ParquetCacheDataConnector.scala | 370 -------------- .../cache/TextCacheDataConnector.scala | 311 ------------ .../direct/AvroDirectDataConnector.scala | 132 ----- .../direct/HiveDirectDataConnector.scala | 158 ------ .../KafkaCacheDirectDataConnector.scala | 125 ----- .../StreamingCacheDirectDataConnector.scala | 60 --- .../KafkaStreamingDataConnector.scala | 58 --- .../{ => data}/connector/DataConnector.scala | 6 +- .../data/connector/DataConnectorFactory.scala | 130 +++++ .../connector/cache/CacheDataConnector.scala | 33 ++ .../data/connector/cache/DataCacheable.scala | 86 ++++ .../data/connector/cache/DataUpdatable.scala | 30 ++ .../cache/HiveCacheDataConnector.scala | 351 +++++++++++++ .../cache/TextCacheDataConnector.scala | 311 ++++++++++++ .../direct/AvroDirectDataConnector.scala | 105 ++++ .../direct/DirectDataConnector.scala | 35 ++ .../direct/HiveDirectDataConnector.scala | 142 ++++++ .../KafkaCacheDirectDataConnector.scala | 125 +++++ .../StreamingCacheDirectDataConnector.scala | 60 +++ .../KafkaStreamingDataConnector.scala | 58 +++ .../streaming/StreamingDataConnector.scala | 34 ++ .../measure/data/source/DataSource.scala | 26 + .../data/source/DataSourceFactory.scala | 43 ++ .../griffin/measure/persist/HdfsPersist.scala | 12 +- .../measure/persist/LoggerPersist.scala | 10 +- .../griffin/measure/persist/Persist.scala | 2 +- .../measure/process/BatchDqProcess.scala | 99 ++++ .../DqProcess.scala} | 20 +- .../measure/process/StreamingDqProcess.scala | 42 ++ .../process/engine/DataFrameOprEngine.scala | 74 +++ .../measure/process/engine/DqEngine.scala | 35 ++ .../process/engine/DqEngineFactory.scala | 47 ++ .../measure/process/engine/DqEngines.scala | 74 +++ .../process/engine/SparkRowFormatter.scala | 62 +++ .../process/engine/SparkSqlEngine.scala | 82 +++ .../rules/adaptor/DataFrameOprAdaptor.scala | 36 ++ .../rules/adaptor/GriffinDslAdaptor.scala | 180 +++++++ .../measure/rules/adaptor/RuleAdaptor.scala | 65 +++ .../rules/adaptor/RuleAdaptorGroup.scala | 93 ++++ .../rules/adaptor/SparkSqlAdaptor.scala | 38 ++ .../griffin/measure/rules/dsl/DqType.scala | 58 +++ .../griffin/measure/rules/dsl/DslType.scala | 58 +++ .../measure/rules/dsl/PersistType.scala | 58 +++ .../rules/dsl/analyzer/AccuracyAnalyzer.scala | 37 ++ .../rules/dsl/analyzer/BasicAnalyzer.scala | 49 ++ .../measure/rules/dsl/expr/AliasExpr.scala | 27 + .../dsl/expr/Expr.scala} | 15 +- .../measure/rules/dsl/expr/FunctionExpr.scala | 29 ++ .../measure/rules/dsl/expr/LiteralExpr.scala | 72 +++ .../measure/rules/dsl/expr/LogicalExpr.scala | 129 +++++ .../measure/rules/dsl/expr/MathExpr.scala | 57 +++ .../measure/rules/dsl/expr/SelectExpr.scala | 88 ++++ .../measure/rules/dsl/expr/TreeNode.scala | 45 ++ .../rules/dsl/parser/BasicParser.scala | 291 +++++++++++ .../rules/dsl/parser/GriffinDslParser.scala | 23 + .../step/ConcreteRuleStep.scala} | 14 +- .../measure/rules/step/DfOprStep.scala | 27 + .../measure/rules/step/GriffinDslStep.scala | 27 + .../step/RuleStep.scala} | 12 +- .../measure/rules/step/SparkSqlStep.scala | 28 + .../griffin/measure/utils/TimeUtil.scala | 4 +- measure/src/test/resources/config-test.json | 52 ++ measure/src/test/resources/config-test1.json | 89 ++++ measure/src/test/resources/config-test2.json | 57 +++ measure/src/test/resources/env-test.json | 38 ++ measure/src/test/resources/input.msg | 1 + measure/src/test/resources/output.msg | 1 + .../algo/batch/BatchAccuracyAlgoTest.scala | 4 +- .../algo/batch/BatchProfileAlgoTest.scala | 4 +- .../algo/batch/DataFrameSaveTest.scala | 4 +- .../streaming/StreamingAccuracyAlgoTest.scala | 4 +- .../{ => data}/connector/ConnectorTest.scala | 2 +- .../measure/process/BatchProcessTest.scala | 146 ++++++ .../measure/process/JsonParseTest.scala | 479 ++++++++++++++++++ .../measure/process/JsonToStructs.scala | 85 ++++ .../rules/dsl/parser/BasicParserTest.scala | 209 ++++++++ 88 files changed, 5227 insertions(+), 2481 deletions(-) delete mode 100644 measure/src/main/scala/org/apache/griffin/measure/connector/DataConnectorFactory.scala delete mode 100644 measure/src/main/scala/org/apache/griffin/measure/connector/cache/DataCacheable.scala delete mode 100644 measure/src/main/scala/org/apache/griffin/measure/connector/cache/DfCacheDataConnector.scala delete mode 100644 measure/src/main/scala/org/apache/griffin/measure/connector/cache/HiveCacheDataConnector.scala delete mode 100644 measure/src/main/scala/org/apache/griffin/measure/connector/cache/ParquetCacheDataConnector.scala delete mode 100644 measure/src/main/scala/org/apache/griffin/measure/connector/cache/TextCacheDataConnector.scala delete mode 100644 measure/src/main/scala/org/apache/griffin/measure/connector/direct/AvroDirectDataConnector.scala delete mode 100644 measure/src/main/scala/org/apache/griffin/measure/connector/direct/HiveDirectDataConnector.scala delete mode 100644 measure/src/main/scala/org/apache/griffin/measure/connector/direct/KafkaCacheDirectDataConnector.scala delete mode 100644 measure/src/main/scala/org/apache/griffin/measure/connector/direct/StreamingCacheDirectDataConnector.scala delete mode 100644 measure/src/main/scala/org/apache/griffin/measure/connector/streaming/KafkaStreamingDataConnector.scala rename measure/src/main/scala/org/apache/griffin/measure/{ => data}/connector/DataConnector.scala (91%) create mode 100644 measure/src/main/scala/org/apache/griffin/measure/data/connector/DataConnectorFactory.scala create mode 100644 measure/src/main/scala/org/apache/griffin/measure/data/connector/cache/CacheDataConnector.scala create mode 100644 measure/src/main/scala/org/apache/griffin/measure/data/connector/cache/DataCacheable.scala create mode 100644 measure/src/main/scala/org/apache/griffin/measure/data/connector/cache/DataUpdatable.scala create mode 100644 measure/src/main/scala/org/apache/griffin/measure/data/connector/cache/HiveCacheDataConnector.scala create mode 100644 measure/src/main/scala/org/apache/griffin/measure/data/connector/cache/TextCacheDataConnector.scala create mode 100644 measure/src/main/scala/org/apache/griffin/measure/data/connector/direct/AvroDirectDataConnector.scala create mode 100644 measure/src/main/scala/org/apache/griffin/measure/data/connector/direct/DirectDataConnector.scala create mode 100644 measure/src/main/scala/org/apache/griffin/measure/data/connector/direct/HiveDirectDataConnector.scala create mode 100644 measure/src/main/scala/org/apache/griffin/measure/data/connector/direct/KafkaCacheDirectDataConnector.scala create mode 100644 measure/src/main/scala/org/apache/griffin/measure/data/connector/direct/StreamingCacheDirectDataConnector.scala create mode 100644 measure/src/main/scala/org/apache/griffin/measure/data/connector/streaming/KafkaStreamingDataConnector.scala create mode 100644 measure/src/main/scala/org/apache/griffin/measure/data/connector/streaming/StreamingDataConnector.scala create mode 100644 measure/src/main/scala/org/apache/griffin/measure/data/source/DataSource.scala create mode 100644 measure/src/main/scala/org/apache/griffin/measure/data/source/DataSourceFactory.scala create mode 100644 measure/src/main/scala/org/apache/griffin/measure/process/BatchDqProcess.scala rename measure/src/main/scala/org/apache/griffin/measure/{connector/direct/DirectDataConnector.scala => process/DqProcess.scala} (66%) create mode 100644 measure/src/main/scala/org/apache/griffin/measure/process/StreamingDqProcess.scala create mode 100644 measure/src/main/scala/org/apache/griffin/measure/process/engine/DataFrameOprEngine.scala create mode 100644 measure/src/main/scala/org/apache/griffin/measure/process/engine/DqEngine.scala create mode 100644 measure/src/main/scala/org/apache/griffin/measure/process/engine/DqEngineFactory.scala create mode 100644 measure/src/main/scala/org/apache/griffin/measure/process/engine/DqEngines.scala create mode 100644 measure/src/main/scala/org/apache/griffin/measure/process/engine/SparkRowFormatter.scala create mode 100644 measure/src/main/scala/org/apache/griffin/measure/process/engine/SparkSqlEngine.scala create mode 100644 measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/DataFrameOprAdaptor.scala create mode 100644 measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/GriffinDslAdaptor.scala create mode 100644 measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/RuleAdaptor.scala create mode 100644 measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/RuleAdaptorGroup.scala create mode 100644 measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/SparkSqlAdaptor.scala create mode 100644 measure/src/main/scala/org/apache/griffin/measure/rules/dsl/DqType.scala create mode 100644 measure/src/main/scala/org/apache/griffin/measure/rules/dsl/DslType.scala create mode 100644 measure/src/main/scala/org/apache/griffin/measure/rules/dsl/PersistType.scala create mode 100644 measure/src/main/scala/org/apache/griffin/measure/rules/dsl/analyzer/AccuracyAnalyzer.scala create mode 100644 measure/src/main/scala/org/apache/griffin/measure/rules/dsl/analyzer/BasicAnalyzer.scala create mode 100644 measure/src/main/scala/org/apache/griffin/measure/rules/dsl/expr/AliasExpr.scala rename measure/src/main/scala/org/apache/griffin/measure/{connector/streaming/StreamingDataConnector.scala => rules/dsl/expr/Expr.scala} (71%) create mode 100644 measure/src/main/scala/org/apache/griffin/measure/rules/dsl/expr/FunctionExpr.scala create mode 100644 measure/src/main/scala/org/apache/griffin/measure/rules/dsl/expr/LiteralExpr.scala create mode 100644 measure/src/main/scala/org/apache/griffin/measure/rules/dsl/expr/LogicalExpr.scala create mode 100644 measure/src/main/scala/org/apache/griffin/measure/rules/dsl/expr/MathExpr.scala create mode 100644 measure/src/main/scala/org/apache/griffin/measure/rules/dsl/expr/SelectExpr.scala create mode 100644 measure/src/main/scala/org/apache/griffin/measure/rules/dsl/expr/TreeNode.scala create mode 100644 measure/src/main/scala/org/apache/griffin/measure/rules/dsl/parser/BasicParser.scala create mode 100644 measure/src/main/scala/org/apache/griffin/measure/rules/dsl/parser/GriffinDslParser.scala rename measure/src/main/scala/org/apache/griffin/measure/{connector/cache/CacheDataConnector.scala => rules/step/ConcreteRuleStep.scala} (68%) create mode 100644 measure/src/main/scala/org/apache/griffin/measure/rules/step/DfOprStep.scala create mode 100644 measure/src/main/scala/org/apache/griffin/measure/rules/step/GriffinDslStep.scala rename measure/src/main/scala/org/apache/griffin/measure/{connector/cache/DataUpdatable.scala => rules/step/RuleStep.scala} (73%) create mode 100644 measure/src/main/scala/org/apache/griffin/measure/rules/step/SparkSqlStep.scala create mode 100644 measure/src/test/resources/config-test.json create mode 100644 measure/src/test/resources/config-test1.json create mode 100644 measure/src/test/resources/config-test2.json create mode 100644 measure/src/test/resources/env-test.json create mode 100644 measure/src/test/resources/input.msg create mode 100644 measure/src/test/resources/output.msg rename measure/src/test/scala/org/apache/griffin/measure/{ => data}/connector/ConnectorTest.scala (98%) create mode 100644 measure/src/test/scala/org/apache/griffin/measure/process/BatchProcessTest.scala create mode 100644 measure/src/test/scala/org/apache/griffin/measure/process/JsonParseTest.scala create mode 100644 measure/src/test/scala/org/apache/griffin/measure/process/JsonToStructs.scala create mode 100644 measure/src/test/scala/org/apache/griffin/measure/rules/dsl/parser/BasicParserTest.scala diff --git a/measure/src/main/resources/config.json b/measure/src/main/resources/config.json index 502535b64..b6e5af995 100644 --- a/measure/src/main/resources/config.json +++ b/measure/src/main/resources/config.json @@ -34,11 +34,26 @@ ], "evaluateRule": { - "dsl.type": "griffin-dsl", "rules": [ { - "type": "accuracy", - "rule": "$source.user_id = $target.user_id AND $source.first_name = $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" + "dsl.type": "griffin-dsl", + "dq.type": "accuracy", + "rule": "source.user_id = target.user_id AND source.first_name = 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", + "miss.record": { + "name": "miss.record", + "persist.type": "record" + }, + "miss.count": { + "name": "miss.count", + "persist.type": "metric" + }, + "total.count": { + "name": "total.count", + "persist.type": "metric" + } + } } ] } diff --git a/measure/src/main/scala/org/apache/griffin/measure/Application.scala b/measure/src/main/scala/org/apache/griffin/measure/Application.scala index 4c322d3b9..5e7e8dbc0 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/Application.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/Application.scala @@ -1,88 +1,141 @@ -///* -//Licensed to the Apache Software Foundation (ASF) under one -//or more contributor license agreements. See the NOTICE file -//distributed with this work for additional information -//regarding copyright ownership. The ASF licenses this file -//to you under the Apache License, Version 2.0 (the -//"License"); you may not use this file except in compliance -//with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -//Unless required by applicable law or agreed to in writing, -//software distributed under the License is distributed on an -//"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -//KIND, either express or implied. See the License for the -//specific language governing permissions and limitations -//under the License. -//*/ -//package org.apache.griffin.measure -// -//import org.apache.griffin.measure.algo._ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ +package org.apache.griffin.measure + +import org.apache.griffin.measure.algo._ //import org.apache.griffin.measure.algo.batch._ //import org.apache.griffin.measure.algo.streaming._ -//import org.apache.griffin.measure.config.params._ -//import org.apache.griffin.measure.config.params.env._ -//import org.apache.griffin.measure.config.params.user._ -//import org.apache.griffin.measure.config.reader._ -//import org.apache.griffin.measure.config.validator.AllParamValidator -//import org.apache.griffin.measure.log.Loggable -//import org.apache.griffin.measure.persist.PersistThreadPool -// -//import scala.util.{Failure, Success, Try} -// -//object Application extends Loggable { -// -// def main(args: Array[String]): Unit = { -// info(args.toString) -// if (args.length < 2) { -// error("Usage: class [List of String split by comma: raw | local | hdfs(default)]") -// sys.exit(-1) -// } -// -// val envParamFile = args(0) -// val userParamFile = args(1) -// val (envFsType, userFsType) = if (args.length > 2) { -// val fsTypes = args(2).trim.split(",") -// if (fsTypes.length == 1) (fsTypes(0).trim, fsTypes(0).trim) -// else if (fsTypes.length >= 2) (fsTypes(0).trim, fsTypes(1).trim) -// else ("hdfs", "hdfs") -// } else ("hdfs", "hdfs") -// -// info(envParamFile) -// info(userParamFile) -// -// // read param files -// val envParam = readParamFile[EnvParam](envParamFile, envFsType) match { -// case Success(p) => p -// case Failure(ex) => { -// error(ex.getMessage) -// sys.exit(-2) -// } -// } -// val userParam = readParamFile[UserParam](userParamFile, userFsType) match { -// case Success(p) => p -// case Failure(ex) => { -// error(ex.getMessage) -// sys.exit(-2) -// } -// } -// val allParam: AllParam = AllParam(envParam, userParam) -// -// // validate param files -// validateParams(allParam) match { -// case Failure(ex) => { -// error(ex.getMessage) -// sys.exit(-3) -// } -// case _ => { -// info("params validation pass") -// } -// } -// -// // choose algorithm +import org.apache.griffin.measure.config.params._ +import org.apache.griffin.measure.config.params.env._ +import org.apache.griffin.measure.config.params.user._ +import org.apache.griffin.measure.config.reader._ +import org.apache.griffin.measure.config.validator.AllParamValidator +import org.apache.griffin.measure.log.Loggable +import org.apache.griffin.measure.persist.PersistThreadPool +import org.apache.griffin.measure.process._ + +import scala.util.{Failure, Success, Try} + +object Application extends Loggable { + + def main(args: Array[String]): Unit = { + info(args.toString) + if (args.length < 2) { + error("Usage: class [List of String split by comma: raw | local | hdfs(default)]") + sys.exit(-1) + } + + val envParamFile = args(0) + val userParamFile = args(1) + val (envFsType, userFsType) = if (args.length > 2) { + val fsTypes = args(2).trim.split(",") + if (fsTypes.length == 1) (fsTypes(0).trim, fsTypes(0).trim) + else if (fsTypes.length >= 2) (fsTypes(0).trim, fsTypes(1).trim) + else ("hdfs", "hdfs") + } else ("hdfs", "hdfs") + + info(envParamFile) + info(userParamFile) + + // read param files + val envParam = readParamFile[EnvParam](envParamFile, envFsType) match { + case Success(p) => p + case Failure(ex) => { + error(ex.getMessage) + sys.exit(-2) + } + } + val userParam = readParamFile[UserParam](userParamFile, userFsType) match { + case Success(p) => p + case Failure(ex) => { + error(ex.getMessage) + sys.exit(-2) + } + } + val allParam: AllParam = AllParam(envParam, userParam) + + // validate param files + validateParams(allParam) match { + case Failure(ex) => { + error(ex.getMessage) + sys.exit(-3) + } + case _ => { + info("params validation pass") + } + } + + // choose algorithm // val dqType = allParam.userParam.dqType -// val procType = allParam.userParam.procType + val procType = allParam.userParam.procType + val proc: DqProcess = procType match { + case ProcessType.batch() => BatchDqProcess(allParam) + case ProcessType.streaming() => StreamingDqProcess(allParam) + case _ => { + error(s"${procType} is unsupported process type!") + sys.exit(-4) + } + } + + // process init + proc.init match { + case Success(_) => { + info("process init success") + } + case Failure(ex) => { + error(s"process init error: ${ex.getMessage}") + shutdown + sys.exit(-5) + } + } + + // process run + proc.run match { + case Success(_) => { + info("process run success") + } + case Failure(ex) => { + error(s"process run error: ${ex.getMessage}") + + if (proc.retriable) { + throw ex + } else { + shutdown + sys.exit(-5) + } + } + } + + // process end + proc.end match { + case Success(_) => { + info("process end success") + } + case Failure(ex) => { + error(s"process end error: ${ex.getMessage}") + shutdown + sys.exit(-5) + } + } + + shutdown + // val algo: Algo = (dqType, procType) match { // case (MeasureType.accuracy(), ProcessType.batch()) => BatchAccuracyAlgo(allParam) // case (MeasureType.profile(), ProcessType.batch()) => BatchProfileAlgo(allParam) @@ -93,8 +146,8 @@ // sys.exit(-4) // } // } -// -// // algorithm run + + // algorithm run // algo.run match { // case Failure(ex) => { // error(s"app error: ${ex.getMessage}") @@ -114,20 +167,20 @@ // info("app finished and success") // } // } -// } -// -// private def readParamFile[T <: Param](file: String, fsType: String)(implicit m : Manifest[T]): Try[T] = { -// val paramReader = ParamReaderFactory.getParamReader(file, fsType) -// paramReader.readConfig[T] -// } -// -// private def validateParams(allParam: AllParam): Try[Boolean] = { -// val allParamValidator = AllParamValidator() -// allParamValidator.validate(allParam) -// } -// -// private def shutdown(): Unit = { -// PersistThreadPool.shutdown -// } -// -//} + } + + private def readParamFile[T <: Param](file: String, fsType: String)(implicit m : Manifest[T]): Try[T] = { + val paramReader = ParamReaderFactory.getParamReader(file, fsType) + paramReader.readConfig[T] + } + + private def validateParams(allParam: AllParam): Try[Boolean] = { + val allParamValidator = AllParamValidator() + allParamValidator.validate(allParam) + } + + private def shutdown(): Unit = { + PersistThreadPool.shutdown + } + +} diff --git a/measure/src/main/scala/org/apache/griffin/measure/algo/batch/BatchAccuracyAlgo.scala b/measure/src/main/scala/org/apache/griffin/measure/algo/batch/BatchAccuracyAlgo.scala index 28366711d..5098d925d 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/algo/batch/BatchAccuracyAlgo.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/algo/batch/BatchAccuracyAlgo.scala @@ -23,8 +23,8 @@ //import org.apache.griffin.measure.algo.AccuracyAlgo //import org.apache.griffin.measure.algo.core.AccuracyCore //import org.apache.griffin.measure.config.params.AllParam -//import org.apache.griffin.measure.connector._ -//import org.apache.griffin.measure.connector.direct.DirectDataConnector +//import org.apache.griffin.measure.data.connector._ +//import org.apache.griffin.measure.data.connector.direct.DirectDataConnector //import org.apache.griffin.measure.persist._ //import org.apache.griffin.measure.result._ //import org.apache.griffin.measure.rule._ diff --git a/measure/src/main/scala/org/apache/griffin/measure/algo/batch/BatchProfileAlgo.scala b/measure/src/main/scala/org/apache/griffin/measure/algo/batch/BatchProfileAlgo.scala index c8c1171ef..53e764804 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/algo/batch/BatchProfileAlgo.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/algo/batch/BatchProfileAlgo.scala @@ -23,8 +23,8 @@ //import org.apache.griffin.measure.algo.ProfileAlgo //import org.apache.griffin.measure.algo.core.ProfileCore //import org.apache.griffin.measure.config.params._ -//import org.apache.griffin.measure.connector._ -//import org.apache.griffin.measure.connector.direct.DirectDataConnector +//import org.apache.griffin.measure.data.connector._ +//import org.apache.griffin.measure.data.connector.direct.DirectDataConnector //import org.apache.griffin.measure.persist.{Persist, PersistFactory, PersistType} //import org.apache.griffin.measure.result._ //import org.apache.griffin.measure.rule.expr._ diff --git a/measure/src/main/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgo.scala b/measure/src/main/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgo.scala index 810356106..9df17c364 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgo.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgo.scala @@ -26,8 +26,8 @@ //import org.apache.griffin.measure.cache.info.{InfoCacheInstance, TimeInfoCache} //import org.apache.griffin.measure.cache.result.CacheResultProcesser //import org.apache.griffin.measure.config.params.AllParam -//import org.apache.griffin.measure.connector._ -//import org.apache.griffin.measure.connector.direct.DirectDataConnector +//import org.apache.griffin.measure.data.connector._ +//import org.apache.griffin.measure.data.connector.direct.DirectDataConnector //import org.apache.griffin.measure.persist.{Persist, PersistFactory, PersistType} //import org.apache.griffin.measure.result.{AccuracyResult, MismatchInfo, TimeStampInfo} //import org.apache.griffin.measure.rule.{ExprValueUtil, RuleAnalyzer, RuleFactory} diff --git a/measure/src/main/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyProcess.scala b/measure/src/main/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyProcess.scala index be1f846d6..1ca8d3449 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyProcess.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyProcess.scala @@ -1,234 +1,234 @@ -/* -Licensed to the Apache Software Foundation (ASF) under one -or more contributor license agreements. See the NOTICE file -distributed with this work for additional information -regarding copyright ownership. The ASF licenses this file -to you under the Apache License, Version 2.0 (the -"License"); you may not use this file except in compliance -with the License. You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, -software distributed under the License is distributed on an -"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -KIND, either express or implied. See the License for the -specific language governing permissions and limitations -under the License. -*/ -package org.apache.griffin.measure.algo.streaming - -import java.util.Date -import java.util.concurrent.TimeUnit - -import org.apache.griffin.measure.algo.core.AccuracyCore -import org.apache.griffin.measure.cache.info.{InfoCacheInstance, TimeInfoCache} -import org.apache.griffin.measure.cache.result.CacheResultProcesser -import org.apache.griffin.measure.connector.direct.DirectDataConnector -import org.apache.griffin.measure.log.Loggable -import org.apache.griffin.measure.persist._ -import org.apache.griffin.measure.result.{AccuracyResult, MismatchInfo, TimeStampInfo} -import org.apache.griffin.measure.rule._ -import org.apache.griffin.measure.rule.expr._ -import org.apache.spark.rdd.RDD - -import scala.util.{Failure, Success} - -case class StreamingAccuracyProcess(sourceDataConnector: DirectDataConnector, - targetDataConnector: DirectDataConnector, - ruleAnalyzer: RuleAnalyzer, - cacheResultProcesser: CacheResultProcesser, - persistFactory: PersistFactory, - appPersist: Persist - ) extends Runnable with Loggable { - - val lock = InfoCacheInstance.genLock("process") - - def run(): Unit = { -// println(s"cache count: ${cacheResultProcesser.cacheGroup.size}") - val updateTimeDate = new Date() - val updateTime = updateTimeDate.getTime - println(s"===== [${updateTimeDate}] process begins =====") - val locked = lock.lock(5, TimeUnit.SECONDS) - if (locked) { - try { - val st = new Date().getTime - - TimeInfoCache.startTimeInfoCache - - // get data - val sourceData = sourceDataConnector.data match { - case Success(dt) => dt - case Failure(ex) => throw ex - } - val targetData = targetDataConnector.data match { - case Success(dt) => dt - case Failure(ex) => throw ex - } - - sourceData.cache - targetData.cache - - println(s"sourceData.count: ${sourceData.count}") - println(s"targetData.count: ${targetData.count}") - - // accuracy algorithm - val (accuResult, missingRdd, matchedRdd) = accuracy(sourceData, targetData, ruleAnalyzer) -// println(s"accuResult: ${accuResult}") - - val ct = new Date().getTime - appPersist.log(ct, s"calculation using time: ${ct - st} ms") - - sourceData.unpersist() - targetData.unpersist() - - // result of every group - val matchedGroups = reorgByTimeGroup(matchedRdd) -// val matchedGroupCount = matchedGroups.count -// println(s"===== matchedGroupCount: ${matchedGroupCount} =====") - - // get missing results - val missingGroups = reorgByTimeGroup(missingRdd) -// val missingGroupCount = missingGroups.count -// println(s"===== missingGroupCount: ${missingGroupCount} =====") - - val groups = matchedGroups.cogroup(missingGroups) -// val groupCount = groups.count -// println(s"===== groupCount: ${groupCount} =====") - - val updateResults = groups.flatMap { group => - val (t, (matchData, missData)) = group - - val matchSize = matchData.size - val missSize = missData.size - val res = AccuracyResult(missSize, matchSize + missSize) - - val updatedCacheResultOpt = cacheResultProcesser.genUpdateCacheResult(t, updateTime, res) - - updatedCacheResultOpt.flatMap { updatedCacheResult => - Some((updatedCacheResult, (t, missData))) - } - } - - updateResults.cache - - val updateResultsPart = updateResults.map(_._1) - val updateDataPart = updateResults.map(_._2) - - val updateResultsArray = updateResultsPart.collect() - - // update results cache (in driver) - // collect action is traversable once action, it will make rdd updateResults empty - updateResultsArray.foreach { updateResult => -// println(s"update result: ${updateResult}") - cacheResultProcesser.update(updateResult) - // persist result - val persist: Persist = persistFactory.getPersists(updateResult.timeGroup) - persist.result(updateTime, updateResult.result) - } - - // record missing data and dump old data (in executor) - updateDataPart.foreach { grp => - val (t, datas) = grp - val persist: Persist = persistFactory.getPersists(t) - // persist missing data - val missStrings = datas.map { row => - record2String(row, ruleAnalyzer.sourceRuleExprs.persistExprs, ruleAnalyzer.targetRuleExprs.persistExprs) - } - persist.records(missStrings, PersistType.MISS) - - // data connector update old data - val dumpDatas = datas.map { r => - val (_, (v, i)) = r - v ++ i - } - sourceDataConnector.updateOldData(t, dumpDatas) -// targetDataConnector.updateOldData(t, dumpDatas) // not correct - } - - updateResults.unpersist() - - TimeInfoCache.endTimeInfoCache - - // clean old data - cleanData() - - val et = new Date().getTime - appPersist.log(et, s"persist using time: ${et - ct} ms") - - } catch { - case e: Throwable => error(s"process error: ${e.getMessage}") - } finally { - lock.unlock() - } - } else { - println(s"===== [${updateTimeDate}] process ignores =====") - } - val endTime = new Date().getTime - println(s"===== [${updateTimeDate}] process ends, using ${endTime - updateTime} ms =====") - } - - // clean old data and old result cache - def cleanData(): Unit = { - try { - sourceDataConnector.cleanOldData - targetDataConnector.cleanOldData - - val cleanTime = TimeInfoCache.getCleanTime - cacheResultProcesser.refresh(cleanTime) - } catch { - case e: Throwable => error(s"clean data error: ${e.getMessage}") - } - } - - // calculate accuracy between source data and target data - private def accuracy(sourceData: RDD[(Product, (Map[String, Any], Map[String, Any]))], - targetData: RDD[(Product, (Map[String, Any], Map[String, Any]))], - ruleAnalyzer: RuleAnalyzer) = { - // 1. cogroup - val allKvs = sourceData.cogroup(targetData) - - // 2. accuracy calculation - val (accuResult, missingRdd, matchedRdd) = AccuracyCore.accuracy(allKvs, ruleAnalyzer) - - (accuResult, missingRdd, matchedRdd) - } - - private def reorgByTimeGroup(rdd: RDD[(Product, (Map[String, Any], Map[String, Any]))] - ): RDD[(Long, (Product, (Map[String, Any], Map[String, Any])))] = { - rdd.flatMap { row => - val (key, (value, info)) = row - val b: Option[(Long, (Product, (Map[String, Any], Map[String, Any])))] = info.get(TimeStampInfo.key) match { - case Some(t: Long) => Some((t, row)) - case _ => None - } - b - } - } - - // convert data into a string - def record2String(rec: (Product, (Map[String, Any], Map[String, Any])), dataPersist: Iterable[Expr], infoPersist: Iterable[Expr]): String = { - val (key, (data, info)) = rec - val persistData = getPersistMap(data, dataPersist) - val persistInfo = info.mapValues { value => - value match { - case vd: Map[String, Any] => getPersistMap(vd, infoPersist) - case v => v - } - }.map(identity) - s"${persistData} [${persistInfo}]" - } - - // get the expr value map of the persist expressions - private def getPersistMap(data: Map[String, Any], persist: Iterable[Expr]): Map[String, Any] = { - val persistMap = persist.map(e => (e._id, e.desc)).toMap - data.flatMap { pair => - val (k, v) = pair - persistMap.get(k) match { - case Some(d) => Some((d -> v)) - case _ => None - } - } - } - -} +///* +//Licensed to the Apache Software Foundation (ASF) under one +//or more contributor license agreements. See the NOTICE file +//distributed with this work for additional information +//regarding copyright ownership. The ASF licenses this file +//to you under the Apache License, Version 2.0 (the +//"License"); you may not use this file except in compliance +//with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +//Unless required by applicable law or agreed to in writing, +//software distributed under the License is distributed on an +//"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +//KIND, either express or implied. See the License for the +//specific language governing permissions and limitations +//under the License. +//*/ +//package org.apache.griffin.measure.algo.streaming +// +//import java.util.Date +//import java.util.concurrent.TimeUnit +// +//import org.apache.griffin.measure.algo.core.AccuracyCore +//import org.apache.griffin.measure.cache.info.{InfoCacheInstance, TimeInfoCache} +//import org.apache.griffin.measure.cache.result.CacheResultProcesser +//import org.apache.griffin.measure.data.connector.direct.DirectDataConnector +//import org.apache.griffin.measure.log.Loggable +//import org.apache.griffin.measure.persist._ +//import org.apache.griffin.measure.result.{AccuracyResult, MismatchInfo, TimeStampInfo} +//import org.apache.griffin.measure.rule._ +//import org.apache.griffin.measure.rule.expr._ +//import org.apache.spark.rdd.RDD +// +//import scala.util.{Failure, Success} +// +//case class StreamingAccuracyProcess(sourceDataConnector: DirectDataConnector, +// targetDataConnector: DirectDataConnector, +// ruleAnalyzer: RuleAnalyzer, +// cacheResultProcesser: CacheResultProcesser, +// persistFactory: PersistFactory, +// appPersist: Persist +// ) extends Runnable with Loggable { +// +// val lock = InfoCacheInstance.genLock("process") +// +// def run(): Unit = { +//// println(s"cache count: ${cacheResultProcesser.cacheGroup.size}") +// val updateTimeDate = new Date() +// val updateTime = updateTimeDate.getTime +// println(s"===== [${updateTimeDate}] process begins =====") +// val locked = lock.lock(5, TimeUnit.SECONDS) +// if (locked) { +// try { +// val st = new Date().getTime +// +// TimeInfoCache.startTimeInfoCache +// +// // get data +// val sourceData = sourceDataConnector.data match { +// case Success(dt) => dt +// case Failure(ex) => throw ex +// } +// val targetData = targetDataConnector.data match { +// case Success(dt) => dt +// case Failure(ex) => throw ex +// } +// +// sourceData.cache +// targetData.cache +// +// println(s"sourceData.count: ${sourceData.count}") +// println(s"targetData.count: ${targetData.count}") +// +// // accuracy algorithm +// val (accuResult, missingRdd, matchedRdd) = accuracy(sourceData, targetData, ruleAnalyzer) +//// println(s"accuResult: ${accuResult}") +// +// val ct = new Date().getTime +// appPersist.log(ct, s"calculation using time: ${ct - st} ms") +// +// sourceData.unpersist() +// targetData.unpersist() +// +// // result of every group +// val matchedGroups = reorgByTimeGroup(matchedRdd) +//// val matchedGroupCount = matchedGroups.count +//// println(s"===== matchedGroupCount: ${matchedGroupCount} =====") +// +// // get missing results +// val missingGroups = reorgByTimeGroup(missingRdd) +//// val missingGroupCount = missingGroups.count +//// println(s"===== missingGroupCount: ${missingGroupCount} =====") +// +// val groups = matchedGroups.cogroup(missingGroups) +//// val groupCount = groups.count +//// println(s"===== groupCount: ${groupCount} =====") +// +// val updateResults = groups.flatMap { group => +// val (t, (matchData, missData)) = group +// +// val matchSize = matchData.size +// val missSize = missData.size +// val res = AccuracyResult(missSize, matchSize + missSize) +// +// val updatedCacheResultOpt = cacheResultProcesser.genUpdateCacheResult(t, updateTime, res) +// +// updatedCacheResultOpt.flatMap { updatedCacheResult => +// Some((updatedCacheResult, (t, missData))) +// } +// } +// +// updateResults.cache +// +// val updateResultsPart = updateResults.map(_._1) +// val updateDataPart = updateResults.map(_._2) +// +// val updateResultsArray = updateResultsPart.collect() +// +// // update results cache (in driver) +// // collect action is traversable once action, it will make rdd updateResults empty +// updateResultsArray.foreach { updateResult => +//// println(s"update result: ${updateResult}") +// cacheResultProcesser.update(updateResult) +// // persist result +// val persist: Persist = persistFactory.getPersists(updateResult.timeGroup) +// persist.result(updateTime, updateResult.result) +// } +// +// // record missing data and dump old data (in executor) +// updateDataPart.foreach { grp => +// val (t, datas) = grp +// val persist: Persist = persistFactory.getPersists(t) +// // persist missing data +// val missStrings = datas.map { row => +// record2String(row, ruleAnalyzer.sourceRuleExprs.persistExprs, ruleAnalyzer.targetRuleExprs.persistExprs) +// } +// persist.records(missStrings, PersistDataType.MISS) +// +// // data connector update old data +// val dumpDatas = datas.map { r => +// val (_, (v, i)) = r +// v ++ i +// } +// sourceDataConnector.updateOldData(t, dumpDatas) +//// targetDataConnector.updateOldData(t, dumpDatas) // not correct +// } +// +// updateResults.unpersist() +// +// TimeInfoCache.endTimeInfoCache +// +// // clean old data +// cleanData() +// +// val et = new Date().getTime +// appPersist.log(et, s"persist using time: ${et - ct} ms") +// +// } catch { +// case e: Throwable => error(s"process error: ${e.getMessage}") +// } finally { +// lock.unlock() +// } +// } else { +// println(s"===== [${updateTimeDate}] process ignores =====") +// } +// val endTime = new Date().getTime +// println(s"===== [${updateTimeDate}] process ends, using ${endTime - updateTime} ms =====") +// } +// +// // clean old data and old result cache +// def cleanData(): Unit = { +// try { +// sourceDataConnector.cleanOldData +// targetDataConnector.cleanOldData +// +// val cleanTime = TimeInfoCache.getCleanTime +// cacheResultProcesser.refresh(cleanTime) +// } catch { +// case e: Throwable => error(s"clean data error: ${e.getMessage}") +// } +// } +// +// // calculate accuracy between source data and target data +// private def accuracy(sourceData: RDD[(Product, (Map[String, Any], Map[String, Any]))], +// targetData: RDD[(Product, (Map[String, Any], Map[String, Any]))], +// ruleAnalyzer: RuleAnalyzer) = { +// // 1. cogroup +// val allKvs = sourceData.cogroup(targetData) +// +// // 2. accuracy calculation +// val (accuResult, missingRdd, matchedRdd) = AccuracyCore.accuracy(allKvs, ruleAnalyzer) +// +// (accuResult, missingRdd, matchedRdd) +// } +// +// private def reorgByTimeGroup(rdd: RDD[(Product, (Map[String, Any], Map[String, Any]))] +// ): RDD[(Long, (Product, (Map[String, Any], Map[String, Any])))] = { +// rdd.flatMap { row => +// val (key, (value, info)) = row +// val b: Option[(Long, (Product, (Map[String, Any], Map[String, Any])))] = info.get(TimeStampInfo.key) match { +// case Some(t: Long) => Some((t, row)) +// case _ => None +// } +// b +// } +// } +// +// // convert data into a string +// def record2String(rec: (Product, (Map[String, Any], Map[String, Any])), dataPersist: Iterable[Expr], infoPersist: Iterable[Expr]): String = { +// val (key, (data, info)) = rec +// val persistData = getPersistMap(data, dataPersist) +// val persistInfo = info.mapValues { value => +// value match { +// case vd: Map[String, Any] => getPersistMap(vd, infoPersist) +// case v => v +// } +// }.map(identity) +// s"${persistData} [${persistInfo}]" +// } +// +// // get the expr value map of the persist expressions +// private def getPersistMap(data: Map[String, Any], persist: Iterable[Expr]): Map[String, Any] = { +// val persistMap = persist.map(e => (e._id, e.desc)).toMap +// data.flatMap { pair => +// val (k, v) = pair +// persistMap.get(k) match { +// case Some(d) => Some((d -> v)) +// case _ => None +// } +// } +// } +// +//} diff --git a/measure/src/main/scala/org/apache/griffin/measure/config/params/user/EvaluateRuleParam.scala b/measure/src/main/scala/org/apache/griffin/measure/config/params/user/EvaluateRuleParam.scala index eddb435cc..2abf3e5d2 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/config/params/user/EvaluateRuleParam.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/config/params/user/EvaluateRuleParam.scala @@ -24,7 +24,7 @@ import org.apache.griffin.measure.config.params.Param @JsonInclude(Include.NON_NULL) case class EvaluateRuleParam( @JsonProperty("dsl.type") dslType: String, - @JsonProperty("rules") rules: List[RuleParam] + @JsonProperty("rules") rules: List[Map[String, Any]] ) extends Param { } diff --git a/measure/src/main/scala/org/apache/griffin/measure/config/params/user/RuleParam.scala b/measure/src/main/scala/org/apache/griffin/measure/config/params/user/RuleParam.scala index 6ce2ac162..ccce71326 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/config/params/user/RuleParam.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/config/params/user/RuleParam.scala @@ -23,10 +23,12 @@ import com.fasterxml.jackson.annotation.JsonInclude.Include import org.apache.griffin.measure.config.params.Param @JsonInclude(Include.NON_NULL) -case class RuleParam( @JsonProperty("name") name: String, - @JsonProperty("type") ruleType: String, - @JsonProperty("rule") procType: String, - @JsonProperty("persist.type") persistType: String +case class RuleParam(@JsonProperty("name") name: String, + @JsonProperty("dsl.type") dslType: String, + @JsonProperty("rule") rule: String, + @JsonProperty("persist.type") persistType: String, + @JsonProperty("dq.type") dqType: String, + @JsonProperty("details") details: Map[String, Any] ) extends Param { } diff --git a/measure/src/main/scala/org/apache/griffin/measure/connector/DataConnectorFactory.scala b/measure/src/main/scala/org/apache/griffin/measure/connector/DataConnectorFactory.scala deleted file mode 100644 index 670175d2f..000000000 --- a/measure/src/main/scala/org/apache/griffin/measure/connector/DataConnectorFactory.scala +++ /dev/null @@ -1,139 +0,0 @@ -/* -Licensed to the Apache Software Foundation (ASF) under one -or more contributor license agreements. See the NOTICE file -distributed with this work for additional information -regarding copyright ownership. The ASF licenses this file -to you under the Apache License, Version 2.0 (the -"License"); you may not use this file except in compliance -with the License. You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, -software distributed under the License is distributed on an -"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -KIND, either express or implied. See the License for the -specific language governing permissions and limitations -under the License. -*/ -package org.apache.griffin.measure.connector - -import kafka.serializer.StringDecoder -import org.apache.griffin.measure.config.params.user._ -import org.apache.griffin.measure.connector.cache._ -import org.apache.griffin.measure.connector.direct._ -import org.apache.griffin.measure.connector.streaming._ -import org.apache.griffin.measure.rule.RuleExprs -import org.apache.spark.sql.SQLContext -import org.apache.spark.streaming.StreamingContext -import org.apache.spark.streaming.dstream.InputDStream -import org.apache.spark.streaming.kafka.KafkaUtils - -import scala.reflect.ClassTag -import scala.util.Try - -object DataConnectorFactory { - - val HiveRegex = """^(?i)hive$""".r - val AvroRegex = """^(?i)avro$""".r - - val KafkaRegex = """^(?i)kafka$""".r - - val TextRegex = """^(?i)text$""".r - - def getDirectDataConnector(sqlContext: SQLContext, - ssc: StreamingContext, - dataConnectorParam: DataConnectorParam, - ruleExprs: RuleExprs, - globalFinalCacheMap: Map[String, Any] - ): Try[DirectDataConnector] = { - val conType = dataConnectorParam.conType - val version = dataConnectorParam.version - val config = dataConnectorParam.config - Try { - conType match { - case HiveRegex() => HiveDirectDataConnector(sqlContext, config, ruleExprs, globalFinalCacheMap) - case AvroRegex() => AvroDirectDataConnector(sqlContext, config, ruleExprs, globalFinalCacheMap) - case KafkaRegex() => { - val ksdcTry = getStreamingDataConnector(ssc, dataConnectorParam) - val cdcTry = getCacheDataConnector(sqlContext, dataConnectorParam.cache) - KafkaCacheDirectDataConnector(ksdcTry, cdcTry, dataConnectorParam, ruleExprs, globalFinalCacheMap) - } - case _ => throw new Exception("connector creation error!") - } - } - } - - private def getStreamingDataConnector(ssc: StreamingContext, - dataConnectorParam: DataConnectorParam - ): Try[StreamingDataConnector] = { - val conType = dataConnectorParam.conType - val version = dataConnectorParam.version - val config = dataConnectorParam.config - Try { - conType match { - case KafkaRegex() => { - genKafkaDataConnector(ssc, config) - } - case _ => throw new Exception("streaming connector creation error!") - } - } - } - - private def getCacheDataConnector(sqlContext: SQLContext, - dataCacheParam: DataCacheParam - ): Try[CacheDataConnector] = { - if (dataCacheParam == null) { - throw new Exception("invalid data cache param!") - } - val cacheType = dataCacheParam.cacheType - Try { - cacheType match { - case HiveRegex() => HiveCacheDataConnector(sqlContext, dataCacheParam) - case TextRegex() => TextCacheDataConnector(sqlContext, dataCacheParam) - case _ => throw new Exception("cache connector creation error!") - } - } - } - - private def genKafkaDataConnector(ssc: StreamingContext, config: Map[String, Any]) = { - val KeyType = "key.type" - val ValueType = "value.type" - val keyType = config.getOrElse(KeyType, "java.lang.String").toString - val valueType = config.getOrElse(ValueType, "java.lang.String").toString -// val KafkaConfig = "kafka.config" -// val Topics = "topics" -// val kafkaConfig = config.get(KafkaConfig) match { -// case Some(map: Map[String, Any]) => map.mapValues(_.toString).map(identity) -// case _ => Map[String, String]() -// } -// val topics = config.getOrElse(Topics, "").toString - (getClassTag(keyType), getClassTag(valueType)) match { - case (ClassTag(k: Class[String]), ClassTag(v: Class[String])) => { - if (ssc == null) throw new Exception("streaming context is null! ") - new KafkaStreamingDataConnector(ssc, config) { - type K = String - type KD = StringDecoder - type V = String - type VD = StringDecoder - def createDStream(topicSet: Set[String]): InputDStream[(K, V)] = { - KafkaUtils.createDirectStream[K, V, KD, VD](ssc, kafkaConfig, topicSet) - } - } - } - case _ => { - throw new Exception("not supported type kafka data connector") - } - } - } - - private def getClassTag(tp: String): ClassTag[_] = { - try { - val clazz = Class.forName(tp) - ClassTag(clazz) - } catch { - case e: Throwable => throw e - } - } - -} diff --git a/measure/src/main/scala/org/apache/griffin/measure/connector/cache/DataCacheable.scala b/measure/src/main/scala/org/apache/griffin/measure/connector/cache/DataCacheable.scala deleted file mode 100644 index 2be87a67b..000000000 --- a/measure/src/main/scala/org/apache/griffin/measure/connector/cache/DataCacheable.scala +++ /dev/null @@ -1,86 +0,0 @@ -/* -Licensed to the Apache Software Foundation (ASF) under one -or more contributor license agreements. See the NOTICE file -distributed with this work for additional information -regarding copyright ownership. The ASF licenses this file -to you under the Apache License, Version 2.0 (the -"License"); you may not use this file except in compliance -with the License. You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, -software distributed under the License is distributed on an -"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -KIND, either express or implied. See the License for the -specific language governing permissions and limitations -under the License. -*/ -package org.apache.griffin.measure.connector.cache - -import java.util.concurrent.atomic.AtomicLong - -import org.apache.griffin.measure.cache.info.{InfoCacheInstance, TimeInfoCache} - -trait DataCacheable { - - protected val defCacheInfoPath = PathCounter.genPath - - val cacheInfoPath: String - val readyTimeInterval: Long - val readyTimeDelay: Long - - def selfCacheInfoPath = s"${TimeInfoCache.infoPath}/${cacheInfoPath}" - - def selfCacheTime = TimeInfoCache.cacheTime(selfCacheInfoPath) - def selfLastProcTime = TimeInfoCache.lastProcTime(selfCacheInfoPath) - def selfReadyTime = TimeInfoCache.readyTime(selfCacheInfoPath) - def selfCleanTime = TimeInfoCache.cleanTime(selfCacheInfoPath) - - protected def submitCacheTime(ms: Long): Unit = { - val map = Map[String, String]((selfCacheTime -> ms.toString)) - InfoCacheInstance.cacheInfo(map) - } - - protected def submitReadyTime(ms: Long): Unit = { - val curReadyTime = ms - readyTimeDelay - if (curReadyTime % readyTimeInterval == 0) { - val map = Map[String, String]((selfReadyTime -> curReadyTime.toString)) - InfoCacheInstance.cacheInfo(map) - } - } - - protected def submitLastProcTime(ms: Long): Unit = { - val map = Map[String, String]((selfLastProcTime -> ms.toString)) - InfoCacheInstance.cacheInfo(map) - } - - protected def submitCleanTime(ms: Long): Unit = { - val cleanTime = genCleanTime(ms) - val map = Map[String, String]((selfCleanTime -> cleanTime.toString)) - InfoCacheInstance.cacheInfo(map) - } - - protected def genCleanTime(ms: Long): Long = ms - - protected def readCleanTime(): Option[Long] = { - val key = selfCleanTime - val keys = key :: Nil - InfoCacheInstance.readInfo(keys).get(key).flatMap { v => - try { - Some(v.toLong) - } catch { - case _ => None - } - } - } - -} - -object PathCounter { - private val counter: AtomicLong = new AtomicLong(0L) - def genPath(): String = s"path_${increment}" - private def increment(): Long = { - counter.incrementAndGet() - } -} \ No newline at end of file diff --git a/measure/src/main/scala/org/apache/griffin/measure/connector/cache/DfCacheDataConnector.scala b/measure/src/main/scala/org/apache/griffin/measure/connector/cache/DfCacheDataConnector.scala deleted file mode 100644 index f883b4fec..000000000 --- a/measure/src/main/scala/org/apache/griffin/measure/connector/cache/DfCacheDataConnector.scala +++ /dev/null @@ -1,281 +0,0 @@ -/* -Licensed to the Apache Software Foundation (ASF) under one -or more contributor license agreements. See the NOTICE file -distributed with this work for additional information -regarding copyright ownership. The ASF licenses this file -to you under the Apache License, Version 2.0 (the -"License"); you may not use this file except in compliance -with the License. You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, -software distributed under the License is distributed on an -"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -KIND, either express or implied. See the License for the -specific language governing permissions and limitations -under the License. -*/ -package org.apache.griffin.measure.connector.cache - -import java.util.concurrent.TimeUnit - -import org.apache.griffin.measure.cache.info.{InfoCacheInstance, TimeInfoCache} -import org.apache.griffin.measure.config.params.user.DataCacheParam -import org.apache.griffin.measure.result.TimeStampInfo -import org.apache.griffin.measure.rule.DataTypeCalculationUtil -import org.apache.griffin.measure.utils.TimeUtil -import org.apache.spark.rdd.RDD -import org.apache.spark.sql.types._ -import org.apache.spark.sql.{DataFrame, Row, SQLContext} -import org.apache.spark.storage.StorageLevel - -import scala.util.Try - -case class DfCacheDataConnector(sqlContext: SQLContext, dataCacheParam: DataCacheParam - ) extends CacheDataConnector { - - val config = dataCacheParam.config - val InfoPath = "info.path" - val cacheInfoPath: String = config.getOrElse(InfoPath, defCacheInfoPath).toString - - val newCacheLock = InfoCacheInstance.genLock(s"${cacheInfoPath}.new") - val oldCacheLock = InfoCacheInstance.genLock(s"${cacheInfoPath}.old") - - val timeRangeParam: List[String] = if (dataCacheParam.timeRange != null) dataCacheParam.timeRange else Nil - val deltaTimeRange: (Long, Long) = (timeRangeParam ::: List("0", "0")) match { - case s :: e :: _ => { - val ns = TimeUtil.milliseconds(s) match { - case Some(n) if (n < 0) => n - case _ => 0 - } - val ne = TimeUtil.milliseconds(e) match { - case Some(n) if (n < 0) => n - case _ => 0 - } - (ns, ne) - } - case _ => (0, 0) - } - - val CacheLevel = "cache.level" - val cacheLevel: String = config.getOrElse(CacheLevel, "MEMORY_AND_DISK").toString - - val timeStampColumn = TimeStampInfo.key - - var newDataFrame: DataFrame = null - var oldDataFrame: DataFrame = null - - val ReadyTimeInterval = "ready.time.interval" - val ReadyTimeDelay = "ready.time.delay" - val readyTimeInterval: Long = TimeUtil.milliseconds(config.getOrElse(ReadyTimeInterval, "1m").toString).getOrElse(60000L) - val readyTimeDelay: Long = TimeUtil.milliseconds(config.getOrElse(ReadyTimeDelay, "1m").toString).getOrElse(60000L) - - def available(): Boolean = { - true - } - - def init(): Unit = {} - - def saveData(rdd: RDD[Map[String, Any]], ms: Long): Unit = { - val newCacheLocked = newCacheLock.lock(-1, TimeUnit.SECONDS) - if (newCacheLocked) { - try { - if (newDataFrame == null) { - if (!rdd.isEmpty) { - newDataFrame = genDataFrame(rdd) - newDataFrame.persist(StorageLevel.fromString(cacheLevel)) - } - } else { - if (!rdd.isEmpty) { - newDataFrame.unpersist() - newDataFrame = newDataFrame.unionAll(genDataFrame(rdd)) - newDataFrame.persist(StorageLevel.fromString(cacheLevel)) - } - } - // submit ms - submitCacheTime(ms) - submitReadyTime(ms) - } catch { - case e: Throwable => error(s"save data error: ${e.getMessage}") - } finally { - newCacheLock.unlock() - } - } - } - - def readData(): Try[RDD[Map[String, Any]]] = Try { - val timeRange = TimeInfoCache.getTimeRange - submitLastProcTime(timeRange._2) - val reviseTimeRange = (timeRange._1 + deltaTimeRange._1, timeRange._2 + deltaTimeRange._2) - - // move new data frame to temp data frame - val newCacheLocked = newCacheLock.lock(-1, TimeUnit.SECONDS) - val newTempDataFrame = if (newCacheLocked) { - try { - val tmp = newDataFrame.filter(s"${timeStampColumn} BETWEEN ${reviseTimeRange._1} AND ${reviseTimeRange._2}") - newDataFrame.unpersist() - newDataFrame = newDataFrame.filter(s"${timeStampColumn} > ${reviseTimeRange._2}") - tmp - } catch { - case _ => null - } finally { - newCacheLock.unlock() - } - } else null - - // add temp data frame to old data frame - val oldCacheLocked = oldCacheLock.lock(-1, TimeUnit.SECONDS) - val oldTempDataFrame = if (oldCacheLocked) { - try { - if (oldDataFrame != null) { - oldDataFrame.filter(s"${timeStampColumn} BETWEEN ${reviseTimeRange._1} AND ${reviseTimeRange._2}") - } else null - } catch { - case _ => null - } finally { - oldCacheLock.unlock() - } - } else null - - val resultDataFrame = if (oldTempDataFrame == null && newTempDataFrame == null) { - throw new Exception("data not cached") - } else { - val finalDataFrame = if (newTempDataFrame == null) { - oldTempDataFrame - } else if (oldTempDataFrame == null) { - newTempDataFrame - } else { - oldTempDataFrame.unionAll(newTempDataFrame) - } - finalDataFrame - } - - // data frame -> rdd - resultDataFrame.map { row => - SparkRowFormatter.formatRow(row) - } - } - - override def cleanOldData(): Unit = { - val oldCacheLocked = oldCacheLock.lock(-1, TimeUnit.SECONDS) - if (oldCacheLocked) { - try { - val timeRange = TimeInfoCache.getTimeRange - val reviseTimeRange = (timeRange._1 + deltaTimeRange._1, timeRange._2 + deltaTimeRange._2) - - oldDataFrame.unpersist() - oldDataFrame = oldDataFrame.filter(s"${timeStampColumn} >= ${reviseTimeRange._1}") - oldDataFrame.persist(StorageLevel.fromString(cacheLevel)) - } catch { - case e: Throwable => error(s"clean old data error: ${e.getMessage}") - } finally { - oldCacheLock.unlock() - } - } - -// if (initialed) { -// val timeRange = TimeInfoCache.getTimeRange -// val reviseTimeRange = (timeRange._1 + deltaTimeRange._1, timeRange._2 + deltaTimeRange._2) -// println(s"clean reviseTimeRange: ${reviseTimeRange}") -// -// dataFrame.show(10) -// -// dataFrame.unpersist() -// dataFrame = dataFrame.filter(s"${timeStampColumn} >= ${reviseTimeRange._1}") -// dataFrame.persist(StorageLevel.fromString(cacheLevel)) -// -// dataFrame.show(10) -// } - } - - override def updateAllOldData(oldRdd: RDD[Map[String, Any]]): Unit = { - val oldCacheLocked = oldCacheLock.lock(-1, TimeUnit.SECONDS) - if (oldCacheLocked) { - try { - if (oldDataFrame == null) { - if (!oldRdd.isEmpty) { - oldDataFrame = genDataFrame(oldRdd) - oldDataFrame.persist(StorageLevel.fromString(cacheLevel)) - } - } else { - if (!oldRdd.isEmpty) { - oldDataFrame.unpersist() - oldDataFrame = genDataFrame(oldRdd) - oldDataFrame.persist(StorageLevel.fromString(cacheLevel)) - } else { - oldDataFrame.unpersist() - oldDataFrame = null - } - } - } catch { - case e: Throwable => error(s"update all old data error: ${e.getMessage}") - } finally { - oldCacheLock.unlock() - } - } - } - - // generate DataFrame - // maybe we can directly use def createDataFrame[A <: Product : TypeTag](rdd: RDD[A]): DataFrame - // to avoid generate data type by myself, just translate each value into Product - private def genDataFrame(rdd: RDD[Map[String, Any]]): DataFrame = { - val fields = rdd.aggregate(Map[String, DataType]())( - DataTypeCalculationUtil.sequenceDataTypeMap, DataTypeCalculationUtil.combineDataTypeMap - ).toList.map(f => StructField(f._1, f._2)) - val schema = StructType(fields) - val datas: RDD[Row] = rdd.map { d => - val values = fields.map { field => - val StructField(k, dt, _, _) = field - d.get(k) match { - case Some(v) => v - case _ => null - } - } - Row(values: _*) - } - val df = sqlContext.createDataFrame(datas, schema) - df - } - -} - -import scala.collection.mutable.ArrayBuffer - -object SparkRowFormatter { - - def formatRow(row: Row): Map[String, Any] = { - formatRowWithSchema(row, row.schema) - } - - private def formatRowWithSchema(row: Row, schema: StructType): Map[String, Any] = { - formatStruct(schema.fields, row) - } - - private def formatStruct(schema: Seq[StructField], r: Row) = { - val paired = schema.zip(r.toSeq) - paired.foldLeft(Map[String, Any]())((s, p) => s ++ formatItem(p)) - } - - private def formatItem(p: Pair[StructField, Any]): Map[String, Any] = { - p match { - case (sf, a) => - sf.dataType match { - case ArrayType(et, _) => - Map(sf.name -> (if (a == null) a else formatArray(et, a.asInstanceOf[ArrayBuffer[Any]]))) - case StructType(s) => - Map(sf.name -> (if (a == null) a else formatStruct(s, a.asInstanceOf[Row]))) - case _ => Map(sf.name -> a) - } - } - } - - private def formatArray(et: DataType, arr: ArrayBuffer[Any]): Seq[Any] = { - et match { - case StructType(s) => arr.map(e => formatStruct(s, e.asInstanceOf[Row])) - case ArrayType(t, _) => - arr.map(e => formatArray(t, e.asInstanceOf[ArrayBuffer[Any]])) - case _ => arr - } - } -} \ No newline at end of file diff --git a/measure/src/main/scala/org/apache/griffin/measure/connector/cache/HiveCacheDataConnector.scala b/measure/src/main/scala/org/apache/griffin/measure/connector/cache/HiveCacheDataConnector.scala deleted file mode 100644 index e24118842..000000000 --- a/measure/src/main/scala/org/apache/griffin/measure/connector/cache/HiveCacheDataConnector.scala +++ /dev/null @@ -1,351 +0,0 @@ -/* -Licensed to the Apache Software Foundation (ASF) under one -or more contributor license agreements. See the NOTICE file -distributed with this work for additional information -regarding copyright ownership. The ASF licenses this file -to you under the Apache License, Version 2.0 (the -"License"); you may not use this file except in compliance -with the License. You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, -software distributed under the License is distributed on an -"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -KIND, either express or implied. See the License for the -specific language governing permissions and limitations -under the License. -*/ -package org.apache.griffin.measure.connector.cache - -import java.util.concurrent.TimeUnit - -import org.apache.griffin.measure.cache.info.{InfoCacheInstance, TimeInfoCache} -import org.apache.griffin.measure.config.params.user.DataCacheParam -import org.apache.griffin.measure.result.TimeStampInfo -import org.apache.griffin.measure.utils.{HdfsFileDumpUtil, HdfsUtil, JsonUtil, TimeUtil} -import org.apache.spark.rdd.RDD -import org.apache.spark.sql.SQLContext -import org.apache.spark.sql.hive.HiveContext - -import scala.util.{Success, Try} - -case class HiveCacheDataConnector(sqlContext: SQLContext, dataCacheParam: DataCacheParam - ) extends CacheDataConnector { - - if (!sqlContext.isInstanceOf[HiveContext]) { - throw new Exception("hive context not prepared!") - } - - val config = dataCacheParam.config - val InfoPath = "info.path" - val cacheInfoPath: String = config.getOrElse(InfoPath, defCacheInfoPath).toString - - val newCacheLock = InfoCacheInstance.genLock(s"${cacheInfoPath}.new") - val oldCacheLock = InfoCacheInstance.genLock(s"${cacheInfoPath}.old") - - val timeRangeParam: List[String] = if (dataCacheParam.timeRange != null) dataCacheParam.timeRange else Nil - val deltaTimeRange: (Long, Long) = (timeRangeParam ::: List("0", "0")) match { - case s :: e :: _ => { - val ns = TimeUtil.milliseconds(s) match { - case Some(n) if (n < 0) => n - case _ => 0 - } - val ne = TimeUtil.milliseconds(e) match { - case Some(n) if (n < 0) => n - case _ => 0 - } - (ns, ne) - } - case _ => (0, 0) - } - - val Database = "database" - val database: String = config.getOrElse(Database, "").toString - val TableName = "table.name" - val tableName: String = config.get(TableName) match { - case Some(s: String) if (s.nonEmpty) => s - case _ => throw new Exception("invalid table.name!") - } - val ParentPath = "parent.path" - val parentPath: String = config.get(ParentPath) match { - case Some(s: String) => s - case _ => throw new Exception("invalid parent.path!") - } - val tablePath = HdfsUtil.getHdfsFilePath(parentPath, tableName) - - val concreteTableName = if (dbPrefix) s"${database}.${tableName}" else tableName - - val ReadyTimeInterval = "ready.time.interval" - val ReadyTimeDelay = "ready.time.delay" - val readyTimeInterval: Long = TimeUtil.milliseconds(config.getOrElse(ReadyTimeInterval, "1m").toString).getOrElse(60000L) - val readyTimeDelay: Long = TimeUtil.milliseconds(config.getOrElse(ReadyTimeDelay, "1m").toString).getOrElse(60000L) - - val TimeStampColumn: String = TimeStampInfo.key - val PayloadColumn: String = "payload" - -// type Schema = (Long, String) - val schema: List[(String, String)] = List( - (TimeStampColumn, "bigint"), - (PayloadColumn, "string") - ) - val schemaName = schema.map(_._1) - -// type Partition = (Long, Long) - val partition: List[(String, String, String)] = List( - ("hr", "bigint", "hour"), - ("min", "bigint", "min") - ) - val partitionName = partition.map(_._1) - - private val fieldSep = """|""" - private val rowSep = """\n""" - private val rowSepLiteral = "\n" - - private def dbPrefix(): Boolean = { - database.nonEmpty && !database.equals("default") - } - - private def tableExists(): Boolean = { - Try { - if (dbPrefix) { - sqlContext.tables(database).filter(tableExistsSql).collect.size - } else { - sqlContext.tables().filter(tableExistsSql).collect.size - } - } match { - case Success(s) => s > 0 - case _ => false - } - } - - override def init(): Unit = { - try { - if (tableExists) { - // drop exist table - val dropSql = s"""DROP TABLE ${concreteTableName}""" - sqlContext.sql(dropSql) - } - - val colsSql = schema.map { field => - s"`${field._1}` ${field._2}" - }.mkString(", ") - val partitionsSql = partition.map { partition => - s"`${partition._1}` ${partition._2}" - }.mkString(", ") - val sql = s"""CREATE EXTERNAL TABLE IF NOT EXISTS ${concreteTableName} - |(${colsSql}) PARTITIONED BY (${partitionsSql}) - |ROW FORMAT DELIMITED - |FIELDS TERMINATED BY '${fieldSep}' - |LINES TERMINATED BY '${rowSep}' - |STORED AS TEXTFILE - |LOCATION '${tablePath}'""".stripMargin - sqlContext.sql(sql) - } catch { - case e: Throwable => throw e - } - } - - def available(): Boolean = { - true - } - - private def encode(data: Map[String, Any], ms: Long): Option[List[Any]] = { - try { - Some(schema.map { field => - val (name, _) = field - name match { - case TimeStampColumn => ms - case PayloadColumn => JsonUtil.toJson(data) - case _ => null - } - }) - } catch { - case _ => None - } - } - - private def decode(data: List[Any], updateTimeStamp: Boolean): Option[Map[String, Any]] = { - val dataMap = schemaName.zip(data).toMap - dataMap.get(PayloadColumn) match { - case Some(v: String) => { - try { - val map = JsonUtil.toAnyMap(v) - val resMap = if (updateTimeStamp) { - dataMap.get(TimeStampColumn) match { - case Some(t) => map + (TimeStampColumn -> t) - case _ => map - } - } else map - Some(resMap) - } catch { - case _ => None - } - } - case _ => None - } - } - - def saveData(rdd: RDD[Map[String, Any]], ms: Long): Unit = { - val newCacheLocked = newCacheLock.lock(-1, TimeUnit.SECONDS) - if (newCacheLocked) { - try { - val ptns = getPartition(ms) - val ptnsPath = genPartitionHdfsPath(ptns) - val dirPath = s"${tablePath}/${ptnsPath}" - val fileName = s"${ms}" - val filePath = HdfsUtil.getHdfsFilePath(dirPath, fileName) - - // encode data - val dataRdd: RDD[List[Any]] = rdd.flatMap(encode(_, ms)) - - // save data - val recordRdd: RDD[String] = dataRdd.map { dt => - dt.map(_.toString).mkString(fieldSep) - } - - val dumped = if (!recordRdd.isEmpty) { - HdfsFileDumpUtil.dump(filePath, recordRdd, rowSepLiteral) - } else false - - // add partition - if (dumped) { - val sql = addPartitionSql(concreteTableName, ptns) - sqlContext.sql(sql) - } - - // submit ms - submitCacheTime(ms) - submitReadyTime(ms) - } catch { - case e: Throwable => error(s"save data error: ${e.getMessage}") - } finally { - newCacheLock.unlock() - } - } - } - - def readData(): Try[RDD[Map[String, Any]]] = Try { - val timeRange = TimeInfoCache.getTimeRange - submitLastProcTime(timeRange._2) - - val reviseTimeRange = (timeRange._1 + deltaTimeRange._1, timeRange._2 + deltaTimeRange._2) - submitCleanTime(reviseTimeRange._1) - - // read directly through partition info - val partitionRange = getPartitionRange(reviseTimeRange._1, reviseTimeRange._2) - val sql = selectSql(concreteTableName, partitionRange) - val df = sqlContext.sql(sql) - - // decode data - df.flatMap { row => - val dt = schemaName.map { sn => - row.getAs[Any](sn) - } - decode(dt, true) - } - } - - override def cleanOldData(): Unit = { - val oldCacheLocked = oldCacheLock.lock(-1, TimeUnit.SECONDS) - if (oldCacheLocked) { - try { - val cleanTime = readCleanTime() - cleanTime match { - case Some(ct) => { - // drop partition - val bound = getPartition(ct) - val sql = dropPartitionSql(concreteTableName, bound) - sqlContext.sql(sql) - } - case _ => { - // do nothing - } - } - } catch { - case e: Throwable => error(s"clean old data error: ${e.getMessage}") - } finally { - oldCacheLock.unlock() - } - } - } - - override def updateOldData(t: Long, oldData: Iterable[Map[String, Any]]): Unit = { - // parallel process different time groups, lock is unnecessary - val ptns = getPartition(t) - val ptnsPath = genPartitionHdfsPath(ptns) - val dirPath = s"${tablePath}/${ptnsPath}" - val fileName = s"${t}" - val filePath = HdfsUtil.getHdfsFilePath(dirPath, fileName) - - try { - // remove out time old data - HdfsFileDumpUtil.remove(dirPath, fileName, true) - - // save updated old data - if (oldData.size > 0) { - val recordDatas = oldData.flatMap { dt => - encode(dt, t) - } - val records: Iterable[String] = recordDatas.map { dt => - dt.map(_.toString).mkString(fieldSep) - } - val dumped = HdfsFileDumpUtil.dump(filePath, records, rowSepLiteral) - } - } catch { - case e: Throwable => error(s"update old data error: ${e.getMessage}") - } - } - - override protected def genCleanTime(ms: Long): Long = { - val minPartition = partition.last - val t1 = TimeUtil.timeToUnit(ms, minPartition._3) - val t2 = TimeUtil.timeFromUnit(t1, minPartition._3) - t2 - } - - private def getPartition(ms: Long): List[(String, Any)] = { - partition.map { p => - val (name, _, unit) = p - val t = TimeUtil.timeToUnit(ms, unit) - (name, t) - } - } - private def getPartitionRange(ms1: Long, ms2: Long): List[(String, (Any, Any))] = { - partition.map { p => - val (name, _, unit) = p - val t1 = TimeUtil.timeToUnit(ms1, unit) - val t2 = TimeUtil.timeToUnit(ms2, unit) - (name, (t1, t2)) - } - } - - private def genPartitionHdfsPath(partition: List[(String, Any)]): String = { - partition.map(prtn => s"${prtn._1}=${prtn._2}").mkString("/") - } - private def addPartitionSql(tbn: String, partition: List[(String, Any)]): String = { - val partitionSql = partition.map(ptn => (s"`${ptn._1}` = ${ptn._2}")).mkString(", ") - val sql = s"""ALTER TABLE ${tbn} ADD IF NOT EXISTS PARTITION (${partitionSql})""" - sql - } - private def selectSql(tbn: String, partitionRange: List[(String, (Any, Any))]): String = { - val clause = partitionRange.map { pr => - val (name, (r1, r2)) = pr - s"""`${name}` BETWEEN '${r1}' and '${r2}'""" - }.mkString(" AND ") - val whereClause = if (clause.nonEmpty) s"WHERE ${clause}" else "" - val sql = s"""SELECT * FROM ${tbn} ${whereClause}""" - sql - } - private def dropPartitionSql(tbn: String, partition: List[(String, Any)]): String = { - val partitionSql = partition.map(ptn => (s"PARTITION ( `${ptn._1}` < '${ptn._2}' ) ")).mkString(", ") - val sql = s"""ALTER TABLE ${tbn} DROP ${partitionSql}""" - println(sql) - sql - } - - private def tableExistsSql(): String = { - s"tableName LIKE '${tableName}'" - } - -} diff --git a/measure/src/main/scala/org/apache/griffin/measure/connector/cache/ParquetCacheDataConnector.scala b/measure/src/main/scala/org/apache/griffin/measure/connector/cache/ParquetCacheDataConnector.scala deleted file mode 100644 index b641d22fe..000000000 --- a/measure/src/main/scala/org/apache/griffin/measure/connector/cache/ParquetCacheDataConnector.scala +++ /dev/null @@ -1,370 +0,0 @@ -/* -Licensed to the Apache Software Foundation (ASF) under one -or more contributor license agreements. See the NOTICE file -distributed with this work for additional information -regarding copyright ownership. The ASF licenses this file -to you under the Apache License, Version 2.0 (the -"License"); you may not use this file except in compliance -with the License. You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, -software distributed under the License is distributed on an -"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -KIND, either express or implied. See the License for the -specific language governing permissions and limitations -under the License. -*/ -package org.apache.griffin.measure.connector.cache - -import java.util.concurrent.TimeUnit - -import org.apache.griffin.measure.cache.info.{InfoCacheInstance, TimeInfoCache} -import org.apache.griffin.measure.config.params.user.DataCacheParam -import org.apache.griffin.measure.result.TimeStampInfo -import org.apache.griffin.measure.utils.{HdfsUtil, JsonUtil, TimeUtil} -import org.apache.spark.rdd.RDD -import org.apache.spark.sql.{Row, SQLContext} -import org.apache.spark.sql.types._ -import scala.collection.JavaConversions._ - -import scala.util.Try - -case class ParquetCacheDataConnector(sqlContext: SQLContext, dataCacheParam: DataCacheParam - ) extends CacheDataConnector { - - val config = dataCacheParam.config - val InfoPath = "info.path" - val cacheInfoPath: String = config.getOrElse(InfoPath, defCacheInfoPath).toString - - val newCacheLock = InfoCacheInstance.genLock(s"${cacheInfoPath}.new") - val oldCacheLock = InfoCacheInstance.genLock(s"${cacheInfoPath}.old") - - val timeRangeParam: List[String] = if (dataCacheParam.timeRange != null) dataCacheParam.timeRange else Nil - val deltaTimeRange: (Long, Long) = (timeRangeParam ::: List("0", "0")) match { - case s :: e :: _ => { - val ns = TimeUtil.milliseconds(s) match { - case Some(n) if (n < 0) => n - case _ => 0 - } - val ne = TimeUtil.milliseconds(e) match { - case Some(n) if (n < 0) => n - case _ => 0 - } - (ns, ne) - } - case _ => (0, 0) - } - - val FilePath = "file.path" - val filePath: String = config.get(FilePath) match { - case Some(s: String) => s - case _ => throw new Exception("invalid file.path!") - } - - val ReadyTimeInterval = "ready.time.interval" - val ReadyTimeDelay = "ready.time.delay" - val readyTimeInterval: Long = TimeUtil.milliseconds(config.getOrElse(ReadyTimeInterval, "1m").toString).getOrElse(60000L) - val readyTimeDelay: Long = TimeUtil.milliseconds(config.getOrElse(ReadyTimeDelay, "1m").toString).getOrElse(60000L) - - val TimeStampColumn: String = TimeStampInfo.key - val PayloadColumn: String = "payload" - - // cache schema: Long, String - val fields = List[StructField]( - StructField(TimeStampColumn, LongType), - StructField(PayloadColumn, StringType) - ) - val schema = StructType(fields) - -// case class CacheData(time: Long, payload: String) { -// def getTime(): Long = time -// def getPayload(): String = payload -// } - - val partitionUnits: List[String] = List("hour", "min") - - override def init(): Unit = { - // do nothing - } - - def available(): Boolean = { - true - } - - private def encode(data: Map[String, Any], ms: Long): Option[Row] = { - try { - val values = fields.map { field => - val StructField(name, _, _, _) = field - name match { - case TimeStampColumn => ms - case PayloadColumn => JsonUtil.toJson(data) - case _ => null - } - } - Some(Row(values: _*)) - } catch { - case _ => None - } - } - - private def decode(data: Row, updateTimeStamp: Boolean): Option[Map[String, Any]] = { - data.getAs[Any](PayloadColumn) match { - case v: String => { - try { - val map = JsonUtil.toAnyMap(v) - val resMap = if (updateTimeStamp) { - data.getAs[Any](TimeStampColumn) match { - case t: Long => map + (TimeStampColumn -> t) - case _ => map - } - } else map - Some(resMap) - } catch { - case _ => None - } - } - case _ => None - } - } - -// private def encode(data: Map[String, Any], ms: Long): Option[CacheData] = { -// try { -// val json = JsonUtil.toJson(data) -// Some(CacheData(ms, json)) -// } catch { -// case _ => None -// } -// } -// -// private def decode(data: Row, updateTimeStamp: Boolean): Option[Map[String, Any]] = { -// data.getAs[Any]("payload") match { -// case v: String => { -// try { -// val map = JsonUtil.toAnyMap(v) -// val resMap = if (updateTimeStamp) { -// data.getAs[Any]("time") match { -// case t: Long => map + (TimeStampInfo.key -> t) -// case _ => map -// } -// } else map -// Some(resMap) -// } catch { -// case _ => None -// } -// } -// case _ => None -// } -// } - - def saveData(rdd: RDD[Map[String, Any]], ms: Long): Unit = { - val newCacheLocked = newCacheLock.lock(-1, TimeUnit.SECONDS) - if (newCacheLocked) { - try { - if (!rdd.isEmpty) { - val ptns = getPartition(ms) - val ptnsPath = genPartitionHdfsPath(ptns) - val dirPath = s"${filePath}/${ptnsPath}" - val dataFileName = s"${ms}" - val dataFilePath = HdfsUtil.getHdfsFilePath(dirPath, dataFileName) - - // encode data - val dataRdd: RDD[Row] = rdd.flatMap(encode(_, ms)) -// val dataRdd: RDD[CacheData] = rdd.flatMap(encode(_, ms)) - - // generate data frame - val df = sqlContext.createDataFrame(dataRdd, schema) -// val df = sqlContext.createDataFrame(dataRdd) - - // save data frame - df.write.parquet(dataFilePath) - } - // submit ms - submitCacheTime(ms) - submitReadyTime(ms) - } catch { - case e: Throwable => error(s"save data error: ${e.getMessage}") - } finally { - newCacheLock.unlock() - } - } - } - - def readData(): Try[RDD[Map[String, Any]]] = Try { - val timeRange = TimeInfoCache.getTimeRange - submitLastProcTime(timeRange._2) - - val reviseTimeRange = (timeRange._1 + deltaTimeRange._1, timeRange._2 + deltaTimeRange._2) - submitCleanTime(reviseTimeRange._1) - - // read directly through partition info - val partitionRanges = getPartitionRange(reviseTimeRange._1, reviseTimeRange._2) - val alterPartitionRanges = partitionRanges :+ (reviseTimeRange._1, reviseTimeRange._2) - println(alterPartitionRanges) - - // list partition paths - val partitionPaths = listPathsBetweenRanges(filePath :: Nil, alterPartitionRanges) - - if (partitionPaths.isEmpty) { - sqlContext.sparkContext.emptyRDD[Map[String, Any]] - } else { - val df = partitionPaths.map { path => - sqlContext.read.parquet(path) - }.reduce(_ unionAll _) - - // decode data - df.flatMap { row => - decode(row, true) - } - } - } - - override def cleanOldData(): Unit = { - val oldCacheLocked = oldCacheLock.lock(-1, TimeUnit.SECONDS) - if (oldCacheLocked) { - try { - val cleanTime = readCleanTime() - cleanTime match { - case Some(ct) => { - // drop partitions - val bounds = getPartition(ct) - val alterBounds = bounds :+ ct - - // list partition paths - val earlierPaths = listPathsEarlierThanBounds(filePath :: Nil, alterBounds) - - // delete out time data path - earlierPaths.foreach { path => - println(s"delete hdfs path: ${path}") - HdfsUtil.deleteHdfsPath(path) - } - } - case _ => { - // do nothing - } - } - } catch { - case e: Throwable => error(s"clean old data error: ${e.getMessage}") - } finally { - oldCacheLock.unlock() - } - } - } - -// override def updateOldData(t: Long, oldData: Iterable[Map[String, Any]]): Unit = { -// // parallel process different time groups, lock is unnecessary -// val ptns = getPartition(t) -// val ptnsPath = genPartitionHdfsPath(ptns) -// val dirPath = s"${filePath}/${ptnsPath}" -// val dataFileName = s"${t}" -// val dataFilePath = HdfsUtil.getHdfsFilePath(dirPath, dataFileName) -// -// try { -// // remove old data path -// HdfsUtil.deleteHdfsPath(dataFilePath) -// -// // save updated old data -// if (oldData.size > 0) { -// // encode data -// val recordDatas = oldData.flatMap { dt => -// encode(dt, t) -// }.toList -// -//// val rdd = sqlContext.sparkContext.parallelize(recordDatas) -// -// // generate data frame -// val df = sqlContext.createDataFrame(recordDatas, schema) -// -// // save data frame -// df.write.parquet(dataFilePath) -// } -// } catch { -// case e: Throwable => { -// error(s"update old data error: ${e.getMessage}") -// e.printStackTrace() -// } -// } -// } - - override protected def genCleanTime(ms: Long): Long = { - val minPartitionUnit = partitionUnits.last - val t1 = TimeUtil.timeToUnit(ms, minPartitionUnit) - val t2 = TimeUtil.timeFromUnit(t1, minPartitionUnit) - t2 - } - - private def getPartition(ms: Long): List[Long] = { - partitionUnits.map { unit => - TimeUtil.timeToUnit(ms, unit) - } - } - private def getPartitionRange(ms1: Long, ms2: Long): List[(Long, Long)] = { - partitionUnits.map { unit => - val t1 = TimeUtil.timeToUnit(ms1, unit) - val t2 = TimeUtil.timeToUnit(ms2, unit) - (t1, t2) - } - } - - private def genPartitionHdfsPath(partition: List[Long]): String = { - partition.map(prtn => s"${prtn}").mkString("/") - } - - private def str2Long(str: String): Option[Long] = { - try { - Some(str.toLong) - } catch { - case e: Throwable => None - } - } - - private def listPathsBetweenRanges(paths: List[String], - partitionRanges: List[(Long, Long)] - ): List[String] = { - partitionRanges match { - case Nil => paths - case head :: tail => { - val (lb, ub) = head - val curPaths = paths.flatMap { path => - val names = HdfsUtil.listSubPaths(path, "dir").toList - names.filter { name => - str2Long(name) match { - case Some(t) => (t >= lb) && (t <= ub) - case _ => false - } - }.map(HdfsUtil.getHdfsFilePath(path, _)) - } - listPathsBetweenRanges(curPaths, tail) - } - } - } - - private def listPathsEarlierThanBounds(paths: List[String], bounds: List[Long] - ): List[String] = { - bounds match { - case Nil => paths - case head :: tail => { - val earlierPaths = paths.flatMap { path => - val names = HdfsUtil.listSubPaths(path, "dir").toList - names.filter { name => - str2Long(name) match { - case Some(t) => (t < head) - case _ => false - } - }.map(HdfsUtil.getHdfsFilePath(path, _)) - } - val equalPaths = paths.flatMap { path => - val names = HdfsUtil.listSubPaths(path, "dir").toList - names.filter { name => - str2Long(name) match { - case Some(t) => (t == head) - case _ => false - } - }.map(HdfsUtil.getHdfsFilePath(path, _)) - } - earlierPaths ::: listPathsEarlierThanBounds(equalPaths, tail) - } - } - } -} diff --git a/measure/src/main/scala/org/apache/griffin/measure/connector/cache/TextCacheDataConnector.scala b/measure/src/main/scala/org/apache/griffin/measure/connector/cache/TextCacheDataConnector.scala deleted file mode 100644 index 62b608618..000000000 --- a/measure/src/main/scala/org/apache/griffin/measure/connector/cache/TextCacheDataConnector.scala +++ /dev/null @@ -1,311 +0,0 @@ -/* -Licensed to the Apache Software Foundation (ASF) under one -or more contributor license agreements. See the NOTICE file -distributed with this work for additional information -regarding copyright ownership. The ASF licenses this file -to you under the Apache License, Version 2.0 (the -"License"); you may not use this file except in compliance -with the License. You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, -software distributed under the License is distributed on an -"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -KIND, either express or implied. See the License for the -specific language governing permissions and limitations -under the License. -*/ -package org.apache.griffin.measure.connector.cache - -import java.util.concurrent.TimeUnit - -import org.apache.griffin.measure.cache.info.{InfoCacheInstance, TimeInfoCache} -import org.apache.griffin.measure.config.params.user.DataCacheParam -import org.apache.griffin.measure.result.TimeStampInfo -import org.apache.griffin.measure.utils.{HdfsFileDumpUtil, HdfsUtil, JsonUtil, TimeUtil} -import org.apache.spark.rdd.RDD -import org.apache.spark.sql.SQLContext - -import scala.util.Try - -case class TextCacheDataConnector(sqlContext: SQLContext, dataCacheParam: DataCacheParam - ) extends CacheDataConnector { - - val config = dataCacheParam.config - val InfoPath = "info.path" - val cacheInfoPath: String = config.getOrElse(InfoPath, defCacheInfoPath).toString - - val newCacheLock = InfoCacheInstance.genLock(s"${cacheInfoPath}.new") - val oldCacheLock = InfoCacheInstance.genLock(s"${cacheInfoPath}.old") - - val timeRangeParam: List[String] = if (dataCacheParam.timeRange != null) dataCacheParam.timeRange else Nil - val deltaTimeRange: (Long, Long) = (timeRangeParam ::: List("0", "0")) match { - case s :: e :: _ => { - val ns = TimeUtil.milliseconds(s) match { - case Some(n) if (n < 0) => n - case _ => 0 - } - val ne = TimeUtil.milliseconds(e) match { - case Some(n) if (n < 0) => n - case _ => 0 - } - (ns, ne) - } - case _ => (0, 0) - } - - val FilePath = "file.path" - val filePath: String = config.get(FilePath) match { - case Some(s: String) => s - case _ => throw new Exception("invalid file.path!") - } - - val ReadyTimeInterval = "ready.time.interval" - val ReadyTimeDelay = "ready.time.delay" - val readyTimeInterval: Long = TimeUtil.milliseconds(config.getOrElse(ReadyTimeInterval, "1m").toString).getOrElse(60000L) - val readyTimeDelay: Long = TimeUtil.milliseconds(config.getOrElse(ReadyTimeDelay, "1m").toString).getOrElse(60000L) - -// val TimeStampColumn: String = TimeStampInfo.key -// val PayloadColumn: String = "payload" - - // cache schema: Long, String -// val fields = List[StructField]( -// StructField(TimeStampColumn, LongType), -// StructField(PayloadColumn, StringType) -// ) -// val schema = StructType(fields) - - // case class CacheData(time: Long, payload: String) { - // def getTime(): Long = time - // def getPayload(): String = payload - // } - - private val rowSepLiteral = "\n" - - val partitionUnits: List[String] = List("hour", "min") - - override def init(): Unit = { - // do nothing - } - - def available(): Boolean = { - true - } - - private def encode(data: Map[String, Any], ms: Long): Option[String] = { - try { - val map = data + (TimeStampInfo.key -> ms) - Some(JsonUtil.toJson(map)) - } catch { - case _: Throwable => None - } - } - - private def decode(data: String): Option[Map[String, Any]] = { - try { - Some(JsonUtil.toAnyMap(data)) - } catch { - case _: Throwable => None - } - } - - def saveData(rdd: RDD[Map[String, Any]], ms: Long): Unit = { - val newCacheLocked = newCacheLock.lock(-1, TimeUnit.SECONDS) - if (newCacheLocked) { - try { - val ptns = getPartition(ms) - val ptnsPath = genPartitionHdfsPath(ptns) - val dirPath = s"${filePath}/${ptnsPath}" - val dataFileName = s"${ms}" - val dataFilePath = HdfsUtil.getHdfsFilePath(dirPath, dataFileName) - - // encode data - val dataRdd: RDD[String] = rdd.flatMap(encode(_, ms)) - - // save data - val dumped = if (!dataRdd.isEmpty) { - HdfsFileDumpUtil.dump(dataFilePath, dataRdd, rowSepLiteral) - } else false - - // submit ms - submitCacheTime(ms) - submitReadyTime(ms) - } catch { - case e: Throwable => error(s"save data error: ${e.getMessage}") - } finally { - newCacheLock.unlock() - } - } - } - - def readData(): Try[RDD[Map[String, Any]]] = Try { - val timeRange = TimeInfoCache.getTimeRange - submitLastProcTime(timeRange._2) - - val reviseTimeRange = (timeRange._1 + deltaTimeRange._1, timeRange._2 + deltaTimeRange._2) - submitCleanTime(reviseTimeRange._1) - - // read directly through partition info - val partitionRanges = getPartitionRange(reviseTimeRange._1, reviseTimeRange._2) - println(s"read time ranges: ${reviseTimeRange}") - println(s"read partition ranges: ${partitionRanges}") - - // list partition paths - val partitionPaths = listPathsBetweenRanges(filePath :: Nil, partitionRanges) - - if (partitionPaths.isEmpty) { - sqlContext.sparkContext.emptyRDD[Map[String, Any]] - } else { - val filePaths = partitionPaths.mkString(",") - val rdd = sqlContext.sparkContext.textFile(filePaths) - - // decode data - rdd.flatMap { row => - decode(row) - } - } - } - - override def cleanOldData(): Unit = { - val oldCacheLocked = oldCacheLock.lock(-1, TimeUnit.SECONDS) - if (oldCacheLocked) { - try { - val cleanTime = readCleanTime() - cleanTime match { - case Some(ct) => { - // drop partitions - val bounds = getPartition(ct) - - // list partition paths - val earlierPaths = listPathsEarlierThanBounds(filePath :: Nil, bounds) - - // delete out time data path - earlierPaths.foreach { path => - println(s"delete hdfs path: ${path}") - HdfsUtil.deleteHdfsPath(path) - } - } - case _ => { - // do nothing - } - } - } catch { - case e: Throwable => error(s"clean old data error: ${e.getMessage}") - } finally { - oldCacheLock.unlock() - } - } - } - - override def updateOldData(t: Long, oldData: Iterable[Map[String, Any]]): Unit = { - // parallel process different time groups, lock is unnecessary - val ptns = getPartition(t) - val ptnsPath = genPartitionHdfsPath(ptns) - val dirPath = s"${filePath}/${ptnsPath}" - val dataFileName = s"${t}" - val dataFilePath = HdfsUtil.getHdfsFilePath(dirPath, dataFileName) - - try { - // remove out time old data - HdfsFileDumpUtil.remove(dirPath, dataFileName, true) - - // save updated old data - if (oldData.size > 0) { - val recordDatas = oldData.flatMap { dt => - encode(dt, t) - } - val dumped = HdfsFileDumpUtil.dump(dataFilePath, recordDatas, rowSepLiteral) - } - } catch { - case e: Throwable => error(s"update old data error: ${e.getMessage}") - } - } - - override protected def genCleanTime(ms: Long): Long = { - val minPartitionUnit = partitionUnits.last - val t1 = TimeUtil.timeToUnit(ms, minPartitionUnit) - val t2 = TimeUtil.timeFromUnit(t1, minPartitionUnit) - t2 - } - - private def getPartition(ms: Long): List[Long] = { - partitionUnits.map { unit => - TimeUtil.timeToUnit(ms, unit) - } - } - private def getPartitionRange(ms1: Long, ms2: Long): List[(Long, Long)] = { - partitionUnits.map { unit => - val t1 = TimeUtil.timeToUnit(ms1, unit) - val t2 = TimeUtil.timeToUnit(ms2, unit) - (t1, t2) - } - } - - private def genPartitionHdfsPath(partition: List[Long]): String = { - partition.map(prtn => s"${prtn}").mkString("/") - } - - private def str2Long(str: String): Option[Long] = { - try { - Some(str.toLong) - } catch { - case e: Throwable => None - } - } - - // here the range means [min, max], but the best range should be (min, max] - private def listPathsBetweenRanges(paths: List[String], - partitionRanges: List[(Long, Long)] - ): List[String] = { - partitionRanges match { - case Nil => paths - case head :: tail => { - val (lb, ub) = head - val curPaths = paths.flatMap { path => - val names = HdfsUtil.listSubPaths(path, "dir").toList - names.filter { name => - str2Long(name) match { - case Some(t) => (t >= lb) && (t <= ub) - case _ => false - } - }.map(HdfsUtil.getHdfsFilePath(path, _)) - } - listPathsBetweenRanges(curPaths, tail) - } - } - } - - private def listPathsEarlierThanBounds(paths: List[String], bounds: List[Long] - ): List[String] = { - bounds match { - case Nil => paths - case head :: tail => { - val earlierPaths = paths.flatMap { path => - val names = HdfsUtil.listSubPaths(path, "dir").toList - names.filter { name => - str2Long(name) match { - case Some(t) => (t < head) - case _ => false - } - }.map(HdfsUtil.getHdfsFilePath(path, _)) - } - val equalPaths = paths.flatMap { path => - val names = HdfsUtil.listSubPaths(path, "dir").toList - names.filter { name => - str2Long(name) match { - case Some(t) => (t == head) - case _ => false - } - }.map(HdfsUtil.getHdfsFilePath(path, _)) - } - - tail match { - case Nil => earlierPaths - case _ => earlierPaths ::: listPathsEarlierThanBounds(equalPaths, tail) - } - } - } - } - -} diff --git a/measure/src/main/scala/org/apache/griffin/measure/connector/direct/AvroDirectDataConnector.scala b/measure/src/main/scala/org/apache/griffin/measure/connector/direct/AvroDirectDataConnector.scala deleted file mode 100644 index b45e5a988..000000000 --- a/measure/src/main/scala/org/apache/griffin/measure/connector/direct/AvroDirectDataConnector.scala +++ /dev/null @@ -1,132 +0,0 @@ -/* -Licensed to the Apache Software Foundation (ASF) under one -or more contributor license agreements. See the NOTICE file -distributed with this work for additional information -regarding copyright ownership. The ASF licenses this file -to you under the Apache License, Version 2.0 (the -"License"); you may not use this file except in compliance -with the License. You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, -software distributed under the License is distributed on an -"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -KIND, either express or implied. See the License for the -specific language governing permissions and limitations -under the License. -*/ -package org.apache.griffin.measure.connector.direct - -import org.apache.griffin.measure.result._ -import org.apache.griffin.measure.rule.{ExprValueUtil, RuleExprs} -import org.apache.griffin.measure.utils.HdfsUtil -import org.apache.spark.rdd.RDD -import org.apache.spark.sql.SQLContext - -import scala.util.Try - -// data connector for avro file -case class AvroDirectDataConnector(sqlContext: SQLContext, config: Map[String, Any], - ruleExprs: RuleExprs, constFinalExprValueMap: Map[String, Any] - ) extends DirectDataConnector { - - val FilePath = "file.path" - val FileName = "file.name" - - val filePath = config.getOrElse(FilePath, "").toString - val fileName = config.getOrElse(FileName, "").toString - - val concreteFileFullPath = if (pathPrefix) s"${filePath}${fileName}" else fileName - - private def pathPrefix(): Boolean = { - filePath.nonEmpty - } - - private def fileExist(): Boolean = { - HdfsUtil.existPath(concreteFileFullPath) - } - - def available(): Boolean = { - (!concreteFileFullPath.isEmpty) && fileExist - } - - def init(): Unit = {} - - def metaData(): Try[Iterable[(String, String)]] = { - Try { - val st = sqlContext.read.format("com.databricks.spark.avro").load(concreteFileFullPath).schema - st.fields.map(f => (f.name, f.dataType.typeName)) - } - } - - def data(): Try[RDD[(Product, (Map[String, Any], Map[String, Any]))]] = { - Try { - loadDataFile.flatMap { row => - // generate cache data - val cacheExprValueMaps = ExprValueUtil.genExprValueMaps(Some(row), ruleExprs.cacheExprs, constFinalExprValueMap) - val finalExprValueMaps = ExprValueUtil.updateExprValueMaps(ruleExprs.finalCacheExprs, cacheExprValueMaps) - - // data info - val dataInfoMap: Map[String, Any] = DataInfo.cacheInfoList.map { info => - try { - (info.key -> row.getAs[info.T](info.key)) - } catch { - case e: Throwable => info.defWrap - } - }.toMap - - finalExprValueMaps.flatMap { finalExprValueMap => - val groupbyData: Seq[AnyRef] = ruleExprs.groupbyExprs.flatMap { expr => - expr.calculate(finalExprValueMap) match { - case Some(v) => Some(v.asInstanceOf[AnyRef]) - case _ => None - } - } - val key = toTuple(groupbyData) - - Some((key, (finalExprValueMap, dataInfoMap))) - } - -// val cacheExprValueMap: Map[String, Any] = ruleExprs.cacheExprs.foldLeft(constFinalExprValueMap) { (cachedMap, expr) => -// ExprValueUtil.genExprValueMaps(Some(row), expr, cachedMap) -// } -// val finalExprValueMap = ExprValueUtil.updateExprValueMaps(ruleExprs.finalCacheExprs, cacheExprValueMap) - - // when clause filter data source -// val whenResult = ruleExprs.whenClauseExprOpt match { -// case Some(whenClause) => whenClause.calculate(finalExprValueMap) -// case _ => None -// } -// -// // get groupby data -// whenResult match { -// case Some(false) => None -// case _ => { -// val groupbyData: Seq[AnyRef] = ruleExprs.groupbyExprs.flatMap { expr => -// expr.calculate(finalExprValueMap) match { -// case Some(v) => Some(v.asInstanceOf[AnyRef]) -// case _ => None -// } -// } -// val key = toTuple(groupbyData) -// -// Some((key, finalExprValueMap)) -// } -// } - } - } - } - - private def loadDataFile() = { - sqlContext.read.format("com.databricks.spark.avro").load(concreteFileFullPath) - } - - private def toTuple[A <: AnyRef](as: Seq[A]): Product = { - if (as.size > 0) { - val tupleClass = Class.forName("scala.Tuple" + as.size) - tupleClass.getConstructors.apply(0).newInstance(as: _*).asInstanceOf[Product] - } else None - } - -} diff --git a/measure/src/main/scala/org/apache/griffin/measure/connector/direct/HiveDirectDataConnector.scala b/measure/src/main/scala/org/apache/griffin/measure/connector/direct/HiveDirectDataConnector.scala deleted file mode 100644 index 7de2b02b6..000000000 --- a/measure/src/main/scala/org/apache/griffin/measure/connector/direct/HiveDirectDataConnector.scala +++ /dev/null @@ -1,158 +0,0 @@ -/* -Licensed to the Apache Software Foundation (ASF) under one -or more contributor license agreements. See the NOTICE file -distributed with this work for additional information -regarding copyright ownership. The ASF licenses this file -to you under the Apache License, Version 2.0 (the -"License"); you may not use this file except in compliance -with the License. You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, -software distributed under the License is distributed on an -"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -KIND, either express or implied. See the License for the -specific language governing permissions and limitations -under the License. -*/ -package org.apache.griffin.measure.connector.direct - -import org.apache.griffin.measure.result._ -import org.apache.griffin.measure.rule.{ExprValueUtil, RuleExprs} -import org.apache.spark.rdd.RDD -import org.apache.spark.sql.SQLContext - -import scala.util.{Success, Try} - -// data connector for hive -case class HiveDirectDataConnector(sqlContext: SQLContext, config: Map[String, Any], - ruleExprs: RuleExprs, constFinalExprValueMap: Map[String, Any] - ) extends DirectDataConnector { - - val Database = "database" - val TableName = "table.name" - val Partitions = "partitions" - - val database = config.getOrElse(Database, "").toString - val tableName = config.getOrElse(TableName, "").toString - val partitionsString = config.getOrElse(Partitions, "").toString - - val concreteTableName = if (dbPrefix) s"${database}.${tableName}" else tableName - val partitions = partitionsString.split(";").map(s => s.split(",").map(_.trim)) - - private def dbPrefix(): Boolean = { - database.nonEmpty && !database.equals("default") - } - - def available(): Boolean = { - (!tableName.isEmpty) && { - Try { - if (dbPrefix) { - sqlContext.tables(database).filter(tableExistsSql).collect.size - } else { - sqlContext.tables().filter(tableExistsSql).collect.size - } - } match { - case Success(s) => s > 0 - case _ => false - } - } - } - - def init(): Unit = {} - - def metaData(): Try[Iterable[(String, String)]] = { - Try { - val originRows = sqlContext.sql(metaDataSql).map(r => (r.getString(0), r.getString(1))).collect - val partitionPos: Int = originRows.indexWhere(pair => pair._1.startsWith("# ")) - if (partitionPos < 0) originRows - else originRows.take(partitionPos) - } - } - - def data(): Try[RDD[(Product, (Map[String, Any], Map[String, Any]))]] = { - Try { - sqlContext.sql(dataSql).flatMap { row => - // generate cache data - val cacheExprValueMaps = ExprValueUtil.genExprValueMaps(Some(row), ruleExprs.cacheExprs, constFinalExprValueMap) - val finalExprValueMaps = ExprValueUtil.updateExprValueMaps(ruleExprs.finalCacheExprs, cacheExprValueMaps) - - // data info - val dataInfoMap: Map[String, Any] = DataInfo.cacheInfoList.map { info => - try { - (info.key -> row.getAs[info.T](info.key)) - } catch { - case e: Throwable => info.defWrap - } - }.toMap - - finalExprValueMaps.flatMap { finalExprValueMap => - val groupbyData: Seq[AnyRef] = ruleExprs.groupbyExprs.flatMap { expr => - expr.calculate(finalExprValueMap) match { - case Some(v) => Some(v.asInstanceOf[AnyRef]) - case _ => None - } - } - val key = toTuple(groupbyData) - - Some((key, (finalExprValueMap, dataInfoMap))) - } - - // generate cache data -// val cacheExprValueMap: Map[String, Any] = ruleExprs.cacheExprs.foldLeft(constFinalExprValueMap) { (cachedMap, expr) => -// ExprValueUtil.genExprValueMap(Some(row), expr, cachedMap) -// } -// val finalExprValueMap = ExprValueUtil.updateExprValueMap(ruleExprs.finalCacheExprs, cacheExprValueMap) -// -// // when clause filter data source -// val whenResult = ruleExprs.whenClauseExprOpt match { -// case Some(whenClause) => whenClause.calculate(finalExprValueMap) -// case _ => None -// } -// -// // get groupby data -// whenResult match { -// case Some(false) => None -// case _ => { -// val groupbyData: Seq[AnyRef] = ruleExprs.groupbyExprs.flatMap { expr => -// expr.calculate(finalExprValueMap) match { -// case Some(v) => Some(v.asInstanceOf[AnyRef]) -// case _ => None -// } -// } -// val key = toTuple(groupbyData) -// -// Some((key, finalExprValueMap)) -// } -// } - } - } - } - - private def tableExistsSql(): String = { -// s"SHOW TABLES LIKE '${concreteTableName}'" // this is hive sql, but not work for spark sql - s"tableName LIKE '${tableName}'" - } - - private def metaDataSql(): String = { - s"DESCRIBE ${concreteTableName}" - } - - private def dataSql(): String = { - val clauses = partitions.map { prtn => - val cls = prtn.mkString(" AND ") - if (cls.isEmpty) s"SELECT * FROM ${concreteTableName}" - else s"SELECT * FROM ${concreteTableName} WHERE ${cls}" - } - clauses.mkString(" UNION ALL ") - } - - private def toTuple[A <: AnyRef](as: Seq[A]): Product = { - if (as.size > 0) { - val tupleClass = Class.forName("scala.Tuple" + as.size) - tupleClass.getConstructors.apply(0).newInstance(as: _*).asInstanceOf[Product] - } else None - } - -} diff --git a/measure/src/main/scala/org/apache/griffin/measure/connector/direct/KafkaCacheDirectDataConnector.scala b/measure/src/main/scala/org/apache/griffin/measure/connector/direct/KafkaCacheDirectDataConnector.scala deleted file mode 100644 index d2534cc77..000000000 --- a/measure/src/main/scala/org/apache/griffin/measure/connector/direct/KafkaCacheDirectDataConnector.scala +++ /dev/null @@ -1,125 +0,0 @@ -/* -Licensed to the Apache Software Foundation (ASF) under one -or more contributor license agreements. See the NOTICE file -distributed with this work for additional information -regarding copyright ownership. The ASF licenses this file -to you under the Apache License, Version 2.0 (the -"License"); you may not use this file except in compliance -with the License. You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, -software distributed under the License is distributed on an -"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -KIND, either express or implied. See the License for the -specific language governing permissions and limitations -under the License. -*/ -package org.apache.griffin.measure.connector.direct - -import org.apache.griffin.measure.config.params.user.DataConnectorParam -import org.apache.griffin.measure.connector.DataConnectorFactory -import org.apache.griffin.measure.connector.cache.CacheDataConnector -import org.apache.griffin.measure.connector.streaming.StreamingDataConnector -import org.apache.griffin.measure.result._ -import org.apache.griffin.measure.rule._ -import org.apache.spark.rdd.RDD -import org.apache.spark.sql.SQLContext -import org.apache.spark.streaming.StreamingContext - -import scala.util.{Failure, Success, Try} - -case class KafkaCacheDirectDataConnector(@transient streamingDataConnectorTry: Try[StreamingDataConnector], - cacheDataConnectorTry: Try[CacheDataConnector], - dataConnectorParam: DataConnectorParam, - ruleExprs: RuleExprs, - constFinalExprValueMap: Map[String, Any] - ) extends StreamingCacheDirectDataConnector { - - val cacheDataConnector: CacheDataConnector = cacheDataConnectorTry match { - case Success(cntr) => cntr - case Failure(ex) => throw ex - } - @transient val streamingDataConnector: StreamingDataConnector = streamingDataConnectorTry match { - case Success(cntr) => cntr - case Failure(ex) => throw ex - } - - protected def transform(rdd: RDD[(streamingDataConnector.K, streamingDataConnector.V)], - ms: Long - ): RDD[Map[String, Any]] = { - val dataInfoMap = DataInfo.cacheInfoList.map(_.defWrap).toMap + TimeStampInfo.wrap(ms) - - rdd.flatMap { kv => - val msg = kv._2 - - val cacheExprValueMaps = ExprValueUtil.genExprValueMaps(Some(msg), ruleExprs.cacheExprs, constFinalExprValueMap) - val finalExprValueMaps = ExprValueUtil.updateExprValueMaps(ruleExprs.finalCacheExprs, cacheExprValueMaps) - - finalExprValueMaps.map { vm => - vm ++ dataInfoMap - } - } - } - - def metaData(): Try[Iterable[(String, String)]] = Try { - Map.empty[String, String] - } - - def data(): Try[RDD[(Product, (Map[String, Any], Map[String, Any]))]] = Try { - cacheDataConnector.readData match { - case Success(rdd) => { - rdd.flatMap { row => - val finalExprValueMap = ruleExprs.finalCacheExprs.flatMap { expr => - row.get(expr._id).flatMap { d => - Some((expr._id, d)) - } - }.toMap - - val dataInfoMap: Map[String, Any] = DataInfo.cacheInfoList.map { info => - row.get(info.key) match { - case Some(d) => (info.key -> d) - case _ => info.defWrap - } - }.toMap - - val groupbyData: Seq[AnyRef] = ruleExprs.groupbyExprs.flatMap { expr => - expr.calculate(finalExprValueMap) match { - case Some(v) => Some(v.asInstanceOf[AnyRef]) - case _ => None - } - } - val key = toTuple(groupbyData) - - Some((key, (finalExprValueMap, dataInfoMap))) - } - } - case Failure(ex) => throw ex - } - } - - override def cleanOldData(): Unit = { - cacheDataConnector.cleanOldData - } - - override def updateOldData(t: Long, oldData: Iterable[Map[String, Any]]): Unit = { - if (dataConnectorParam.getMatchOnce) { - cacheDataConnector.updateOldData(t, oldData) - } - } - - override def updateAllOldData(oldRdd: RDD[Map[String, Any]]): Unit = { - if (dataConnectorParam.getMatchOnce) { - cacheDataConnector.updateAllOldData(oldRdd) - } - } - - private def toTuple[A <: AnyRef](as: Seq[A]): Product = { - if (as.size > 0) { - val tupleClass = Class.forName("scala.Tuple" + as.size) - tupleClass.getConstructors.apply(0).newInstance(as: _*).asInstanceOf[Product] - } else None - } - -} diff --git a/measure/src/main/scala/org/apache/griffin/measure/connector/direct/StreamingCacheDirectDataConnector.scala b/measure/src/main/scala/org/apache/griffin/measure/connector/direct/StreamingCacheDirectDataConnector.scala deleted file mode 100644 index 87139d6a4..000000000 --- a/measure/src/main/scala/org/apache/griffin/measure/connector/direct/StreamingCacheDirectDataConnector.scala +++ /dev/null @@ -1,60 +0,0 @@ -/* -Licensed to the Apache Software Foundation (ASF) under one -or more contributor license agreements. See the NOTICE file -distributed with this work for additional information -regarding copyright ownership. The ASF licenses this file -to you under the Apache License, Version 2.0 (the -"License"); you may not use this file except in compliance -with the License. You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, -software distributed under the License is distributed on an -"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -KIND, either express or implied. See the License for the -specific language governing permissions and limitations -under the License. -*/ -package org.apache.griffin.measure.connector.direct - -import org.apache.griffin.measure.connector.cache.CacheDataConnector -import org.apache.griffin.measure.connector.streaming.StreamingDataConnector -import org.apache.griffin.measure.result.{DataInfo, TimeStampInfo} -import org.apache.griffin.measure.rule.ExprValueUtil -import org.apache.spark.rdd.RDD - -import scala.util.{Failure, Success} - -trait StreamingCacheDirectDataConnector extends DirectDataConnector { - - val cacheDataConnector: CacheDataConnector - @transient val streamingDataConnector: StreamingDataConnector - - def available(): Boolean = { - cacheDataConnector.available && streamingDataConnector.available - } - - def init(): Unit = { - cacheDataConnector.init - - val ds = streamingDataConnector.stream match { - case Success(dstream) => dstream - case Failure(ex) => throw ex - } - - ds.foreachRDD((rdd, time) => { - val ms = time.milliseconds - - val valueMapRdd = transform(rdd, ms) - - // save data frame - cacheDataConnector.saveData(valueMapRdd, ms) - }) - } - - protected def transform(rdd: RDD[(streamingDataConnector.K, streamingDataConnector.V)], - ms: Long - ): RDD[Map[String, Any]] - -} diff --git a/measure/src/main/scala/org/apache/griffin/measure/connector/streaming/KafkaStreamingDataConnector.scala b/measure/src/main/scala/org/apache/griffin/measure/connector/streaming/KafkaStreamingDataConnector.scala deleted file mode 100644 index fdd511d26..000000000 --- a/measure/src/main/scala/org/apache/griffin/measure/connector/streaming/KafkaStreamingDataConnector.scala +++ /dev/null @@ -1,58 +0,0 @@ -/* -Licensed to the Apache Software Foundation (ASF) under one -or more contributor license agreements. See the NOTICE file -distributed with this work for additional information -regarding copyright ownership. The ASF licenses this file -to you under the Apache License, Version 2.0 (the -"License"); you may not use this file except in compliance -with the License. You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, -software distributed under the License is distributed on an -"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -KIND, either express or implied. See the License for the -specific language governing permissions and limitations -under the License. -*/ -package org.apache.griffin.measure.connector.streaming - -import kafka.serializer.Decoder -import org.apache.griffin.measure.connector.cache.{CacheDataConnector, DataCacheable} -import org.apache.griffin.measure.result.{DataInfo, TimeStampInfo} -import org.apache.griffin.measure.rule.{ExprValueUtil, RuleExprs} -import org.apache.spark.rdd.RDD -import org.apache.spark.streaming.StreamingContext -import org.apache.spark.streaming.dstream.InputDStream - -import scala.util.{Failure, Success, Try} - -abstract class KafkaStreamingDataConnector(@transient ssc: StreamingContext, - config: Map[String, Any] - ) extends StreamingDataConnector { - type KD <: Decoder[K] - type VD <: Decoder[V] - - val KafkaConfig = "kafka.config" - val Topics = "topics" - - val kafkaConfig = config.get(KafkaConfig) match { - case Some(map: Map[String, Any]) => map.mapValues(_.toString).map(identity) - case _ => Map[String, String]() - } - val topics = config.getOrElse(Topics, "").toString - - def available(): Boolean = { - true - } - - def init(): Unit = {} - - def stream(): Try[InputDStream[(K, V)]] = Try { - val topicSet = topics.split(",").toSet - createDStream(topicSet) - } - - protected def createDStream(topicSet: Set[String]): InputDStream[(K, V)] -} \ No newline at end of file diff --git a/measure/src/main/scala/org/apache/griffin/measure/connector/DataConnector.scala b/measure/src/main/scala/org/apache/griffin/measure/data/connector/DataConnector.scala similarity index 91% rename from measure/src/main/scala/org/apache/griffin/measure/connector/DataConnector.scala rename to measure/src/main/scala/org/apache/griffin/measure/data/connector/DataConnector.scala index 1fb1868fd..a766b42ca 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/connector/DataConnector.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/data/connector/DataConnector.scala @@ -16,7 +16,7 @@ KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License. */ -package org.apache.griffin.measure.connector +package org.apache.griffin.measure.data.connector import org.apache.griffin.measure.log.Loggable import org.apache.spark.rdd.RDD @@ -25,8 +25,6 @@ import org.apache.spark.sql.DataFrame trait DataConnector extends Loggable with Serializable { - def available(): Boolean - - def init(): Unit +// def available(): Boolean } diff --git a/measure/src/main/scala/org/apache/griffin/measure/data/connector/DataConnectorFactory.scala b/measure/src/main/scala/org/apache/griffin/measure/data/connector/DataConnectorFactory.scala new file mode 100644 index 000000000..7ee42a473 --- /dev/null +++ b/measure/src/main/scala/org/apache/griffin/measure/data/connector/DataConnectorFactory.scala @@ -0,0 +1,130 @@ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ +package org.apache.griffin.measure.data.connector + +import kafka.serializer.StringDecoder +import org.apache.griffin.measure.config.params.user._ +//import org.apache.griffin.measure.data.connector.cache._ +import org.apache.griffin.measure.data.connector.direct._ +//import org.apache.griffin.measure.data.connector.streaming._ +import org.apache.griffin.measure.rule.RuleExprs +import org.apache.spark.sql.SQLContext +import org.apache.spark.streaming.StreamingContext +import org.apache.spark.streaming.dstream.InputDStream +import org.apache.spark.streaming.kafka.KafkaUtils + +import scala.reflect.ClassTag +import scala.util.Try + +object DataConnectorFactory { + + val HiveRegex = """^(?i)hive$""".r + val AvroRegex = """^(?i)avro$""".r + + val KafkaRegex = """^(?i)kafka$""".r + + val TextRegex = """^(?i)text$""".r + + def getDirectDataConnector(sqlContext: SQLContext, + ssc: StreamingContext, + dataConnectorParam: DataConnectorParam + ): Try[DirectDataConnector] = { + val conType = dataConnectorParam.conType + val version = dataConnectorParam.version + val config = dataConnectorParam.config + Try { + conType match { + case HiveRegex() => HiveDirectDataConnector(sqlContext, config) + case AvroRegex() => AvroDirectDataConnector(sqlContext, config) +// case KafkaRegex() => { +// val ksdcTry = getStreamingDataConnector(ssc, dataConnectorParam) +// val cdcTry = getCacheDataConnector(sqlContext, dataConnectorParam.cache) +// KafkaCacheDirectDataConnector(ksdcTry, cdcTry, dataConnectorParam) +// } + case _ => throw new Exception("connector creation error!") + } + } + } + +// private def getStreamingDataConnector(ssc: StreamingContext, +// dataConnectorParam: DataConnectorParam +// ): Try[StreamingDataConnector] = { +// val conType = dataConnectorParam.conType +// val version = dataConnectorParam.version +// val config = dataConnectorParam.config +// Try { +// conType match { +// case KafkaRegex() => { +// genKafkaDataConnector(ssc, config) +// } +// case _ => throw new Exception("streaming connector creation error!") +// } +// } +// } +// +// private def getCacheDataConnector(sqlContext: SQLContext, +// dataCacheParam: DataCacheParam +// ): Try[CacheDataConnector] = { +// if (dataCacheParam == null) { +// throw new Exception("invalid data cache param!") +// } +// val cacheType = dataCacheParam.cacheType +// Try { +// cacheType match { +// case HiveRegex() => HiveCacheDataConnector(sqlContext, dataCacheParam) +// case TextRegex() => TextCacheDataConnector(sqlContext, dataCacheParam) +// case _ => throw new Exception("cache connector creation error!") +// } +// } +// } +// +// private def genKafkaDataConnector(ssc: StreamingContext, config: Map[String, Any]) = { +// val KeyType = "key.type" +// val ValueType = "value.type" +// val keyType = config.getOrElse(KeyType, "java.lang.String").toString +// val valueType = config.getOrElse(ValueType, "java.lang.String").toString +// (getClassTag(keyType), getClassTag(valueType)) match { +// case (ClassTag(k: Class[String]), ClassTag(v: Class[String])) => { +// if (ssc == null) throw new Exception("streaming context is null! ") +// new KafkaStreamingDataConnector(ssc, config) { +// type K = String +// type KD = StringDecoder +// type V = String +// type VD = StringDecoder +// def createDStream(topicSet: Set[String]): InputDStream[(K, V)] = { +// KafkaUtils.createDirectStream[K, V, KD, VD](ssc, kafkaConfig, topicSet) +// } +// } +// } +// case _ => { +// throw new Exception("not supported type kafka data connector") +// } +// } +// } +// +// private def getClassTag(tp: String): ClassTag[_] = { +// try { +// val clazz = Class.forName(tp) +// ClassTag(clazz) +// } catch { +// case e: Throwable => throw e +// } +// } + +} diff --git a/measure/src/main/scala/org/apache/griffin/measure/data/connector/cache/CacheDataConnector.scala b/measure/src/main/scala/org/apache/griffin/measure/data/connector/cache/CacheDataConnector.scala new file mode 100644 index 000000000..67dcc065a --- /dev/null +++ b/measure/src/main/scala/org/apache/griffin/measure/data/connector/cache/CacheDataConnector.scala @@ -0,0 +1,33 @@ +///* +//Licensed to the Apache Software Foundation (ASF) under one +//or more contributor license agreements. See the NOTICE file +//distributed with this work for additional information +//regarding copyright ownership. The ASF licenses this file +//to you under the Apache License, Version 2.0 (the +//"License"); you may not use this file except in compliance +//with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +//Unless required by applicable law or agreed to in writing, +//software distributed under the License is distributed on an +//"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +//KIND, either express or implied. See the License for the +//specific language governing permissions and limitations +//under the License. +//*/ +//package org.apache.griffin.measure.data.connector.cache +// +//import org.apache.griffin.measure.data.connector.DataConnector +//import org.apache.spark.rdd.RDD +// +//import scala.util.Try +// +//trait CacheDataConnector extends DataConnector with DataCacheable with DataUpdatable { +// +// def saveData(rdd: RDD[Map[String, Any]], ms: Long): Unit +// +// def readData(): Try[RDD[Map[String, Any]]] +// +//} +// diff --git a/measure/src/main/scala/org/apache/griffin/measure/data/connector/cache/DataCacheable.scala b/measure/src/main/scala/org/apache/griffin/measure/data/connector/cache/DataCacheable.scala new file mode 100644 index 000000000..79162be84 --- /dev/null +++ b/measure/src/main/scala/org/apache/griffin/measure/data/connector/cache/DataCacheable.scala @@ -0,0 +1,86 @@ +///* +//Licensed to the Apache Software Foundation (ASF) under one +//or more contributor license agreements. See the NOTICE file +//distributed with this work for additional information +//regarding copyright ownership. The ASF licenses this file +//to you under the Apache License, Version 2.0 (the +//"License"); you may not use this file except in compliance +//with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +//Unless required by applicable law or agreed to in writing, +//software distributed under the License is distributed on an +//"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +//KIND, either express or implied. See the License for the +//specific language governing permissions and limitations +//under the License. +//*/ +//package org.apache.griffin.measure.data.connector.cache +// +//import java.util.concurrent.atomic.AtomicLong +// +//import org.apache.griffin.measure.cache.info.{InfoCacheInstance, TimeInfoCache} +// +//trait DataCacheable { +// +// protected val defCacheInfoPath = PathCounter.genPath +// +// val cacheInfoPath: String +// val readyTimeInterval: Long +// val readyTimeDelay: Long +// +// def selfCacheInfoPath = s"${TimeInfoCache.infoPath}/${cacheInfoPath}" +// +// def selfCacheTime = TimeInfoCache.cacheTime(selfCacheInfoPath) +// def selfLastProcTime = TimeInfoCache.lastProcTime(selfCacheInfoPath) +// def selfReadyTime = TimeInfoCache.readyTime(selfCacheInfoPath) +// def selfCleanTime = TimeInfoCache.cleanTime(selfCacheInfoPath) +// +// protected def submitCacheTime(ms: Long): Unit = { +// val map = Map[String, String]((selfCacheTime -> ms.toString)) +// InfoCacheInstance.cacheInfo(map) +// } +// +// protected def submitReadyTime(ms: Long): Unit = { +// val curReadyTime = ms - readyTimeDelay +// if (curReadyTime % readyTimeInterval == 0) { +// val map = Map[String, String]((selfReadyTime -> curReadyTime.toString)) +// InfoCacheInstance.cacheInfo(map) +// } +// } +// +// protected def submitLastProcTime(ms: Long): Unit = { +// val map = Map[String, String]((selfLastProcTime -> ms.toString)) +// InfoCacheInstance.cacheInfo(map) +// } +// +// protected def submitCleanTime(ms: Long): Unit = { +// val cleanTime = genCleanTime(ms) +// val map = Map[String, String]((selfCleanTime -> cleanTime.toString)) +// InfoCacheInstance.cacheInfo(map) +// } +// +// protected def genCleanTime(ms: Long): Long = ms +// +// protected def readCleanTime(): Option[Long] = { +// val key = selfCleanTime +// val keys = key :: Nil +// InfoCacheInstance.readInfo(keys).get(key).flatMap { v => +// try { +// Some(v.toLong) +// } catch { +// case _ => None +// } +// } +// } +// +//} +// +//object PathCounter { +// private val counter: AtomicLong = new AtomicLong(0L) +// def genPath(): String = s"path_${increment}" +// private def increment(): Long = { +// counter.incrementAndGet() +// } +//} \ No newline at end of file diff --git a/measure/src/main/scala/org/apache/griffin/measure/data/connector/cache/DataUpdatable.scala b/measure/src/main/scala/org/apache/griffin/measure/data/connector/cache/DataUpdatable.scala new file mode 100644 index 000000000..61e84138e --- /dev/null +++ b/measure/src/main/scala/org/apache/griffin/measure/data/connector/cache/DataUpdatable.scala @@ -0,0 +1,30 @@ +///* +//Licensed to the Apache Software Foundation (ASF) under one +//or more contributor license agreements. See the NOTICE file +//distributed with this work for additional information +//regarding copyright ownership. The ASF licenses this file +//to you under the Apache License, Version 2.0 (the +//"License"); you may not use this file except in compliance +//with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +//Unless required by applicable law or agreed to in writing, +//software distributed under the License is distributed on an +//"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +//KIND, either express or implied. See the License for the +//specific language governing permissions and limitations +//under the License. +//*/ +//package org.apache.griffin.measure.data.connector.cache +// +//import org.apache.spark.rdd.RDD +// +//trait DataUpdatable { +// +// def cleanOldData(): Unit = {} +// +// def updateOldData(t: Long, oldData: Iterable[Map[String, Any]]): Unit = {} +// def updateAllOldData(oldRdd: RDD[Map[String, Any]]): Unit = {} +// +//} diff --git a/measure/src/main/scala/org/apache/griffin/measure/data/connector/cache/HiveCacheDataConnector.scala b/measure/src/main/scala/org/apache/griffin/measure/data/connector/cache/HiveCacheDataConnector.scala new file mode 100644 index 000000000..4c7b45bef --- /dev/null +++ b/measure/src/main/scala/org/apache/griffin/measure/data/connector/cache/HiveCacheDataConnector.scala @@ -0,0 +1,351 @@ +///* +//Licensed to the Apache Software Foundation (ASF) under one +//or more contributor license agreements. See the NOTICE file +//distributed with this work for additional information +//regarding copyright ownership. The ASF licenses this file +//to you under the Apache License, Version 2.0 (the +//"License"); you may not use this file except in compliance +//with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +//Unless required by applicable law or agreed to in writing, +//software distributed under the License is distributed on an +//"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +//KIND, either express or implied. See the License for the +//specific language governing permissions and limitations +//under the License. +//*/ +//package org.apache.griffin.measure.data.connector.cache +// +//import java.util.concurrent.TimeUnit +// +//import org.apache.griffin.measure.cache.info.{InfoCacheInstance, TimeInfoCache} +//import org.apache.griffin.measure.config.params.user.DataCacheParam +//import org.apache.griffin.measure.result.TimeStampInfo +//import org.apache.griffin.measure.utils.{HdfsFileDumpUtil, HdfsUtil, JsonUtil, TimeUtil} +//import org.apache.spark.rdd.RDD +//import org.apache.spark.sql.SQLContext +//import org.apache.spark.sql.hive.HiveContext +// +//import scala.util.{Success, Try} +// +//case class HiveCacheDataConnector(sqlContext: SQLContext, dataCacheParam: DataCacheParam +// ) extends CacheDataConnector { +// +// if (!sqlContext.isInstanceOf[HiveContext]) { +// throw new Exception("hive context not prepared!") +// } +// +// val config = dataCacheParam.config +// val InfoPath = "info.path" +// val cacheInfoPath: String = config.getOrElse(InfoPath, defCacheInfoPath).toString +// +// val newCacheLock = InfoCacheInstance.genLock(s"${cacheInfoPath}.new") +// val oldCacheLock = InfoCacheInstance.genLock(s"${cacheInfoPath}.old") +// +// val timeRangeParam: List[String] = if (dataCacheParam.timeRange != null) dataCacheParam.timeRange else Nil +// val deltaTimeRange: (Long, Long) = (timeRangeParam ::: List("0", "0")) match { +// case s :: e :: _ => { +// val ns = TimeUtil.milliseconds(s) match { +// case Some(n) if (n < 0) => n +// case _ => 0 +// } +// val ne = TimeUtil.milliseconds(e) match { +// case Some(n) if (n < 0) => n +// case _ => 0 +// } +// (ns, ne) +// } +// case _ => (0, 0) +// } +// +// val Database = "database" +// val database: String = config.getOrElse(Database, "").toString +// val TableName = "table.name" +// val tableName: String = config.get(TableName) match { +// case Some(s: String) if (s.nonEmpty) => s +// case _ => throw new Exception("invalid table.name!") +// } +// val ParentPath = "parent.path" +// val parentPath: String = config.get(ParentPath) match { +// case Some(s: String) => s +// case _ => throw new Exception("invalid parent.path!") +// } +// val tablePath = HdfsUtil.getHdfsFilePath(parentPath, tableName) +// +// val concreteTableName = if (dbPrefix) s"${database}.${tableName}" else tableName +// +// val ReadyTimeInterval = "ready.time.interval" +// val ReadyTimeDelay = "ready.time.delay" +// val readyTimeInterval: Long = TimeUtil.milliseconds(config.getOrElse(ReadyTimeInterval, "1m").toString).getOrElse(60000L) +// val readyTimeDelay: Long = TimeUtil.milliseconds(config.getOrElse(ReadyTimeDelay, "1m").toString).getOrElse(60000L) +// +// val TimeStampColumn: String = TimeStampInfo.key +// val PayloadColumn: String = "payload" +// +//// type Schema = (Long, String) +// val schema: List[(String, String)] = List( +// (TimeStampColumn, "bigint"), +// (PayloadColumn, "string") +// ) +// val schemaName = schema.map(_._1) +// +//// type Partition = (Long, Long) +// val partition: List[(String, String, String)] = List( +// ("hr", "bigint", "hour"), +// ("min", "bigint", "min") +// ) +// val partitionName = partition.map(_._1) +// +// private val fieldSep = """|""" +// private val rowSep = """\n""" +// private val rowSepLiteral = "\n" +// +// private def dbPrefix(): Boolean = { +// database.nonEmpty && !database.equals("default") +// } +// +// private def tableExists(): Boolean = { +// Try { +// if (dbPrefix) { +// sqlContext.tables(database).filter(tableExistsSql).collect.size +// } else { +// sqlContext.tables().filter(tableExistsSql).collect.size +// } +// } match { +// case Success(s) => s > 0 +// case _ => false +// } +// } +// +// override def init(): Unit = { +// try { +// if (tableExists) { +// // drop exist table +// val dropSql = s"""DROP TABLE ${concreteTableName}""" +// sqlContext.sql(dropSql) +// } +// +// val colsSql = schema.map { field => +// s"`${field._1}` ${field._2}" +// }.mkString(", ") +// val partitionsSql = partition.map { partition => +// s"`${partition._1}` ${partition._2}" +// }.mkString(", ") +// val sql = s"""CREATE EXTERNAL TABLE IF NOT EXISTS ${concreteTableName} +// |(${colsSql}) PARTITIONED BY (${partitionsSql}) +// |ROW FORMAT DELIMITED +// |FIELDS TERMINATED BY '${fieldSep}' +// |LINES TERMINATED BY '${rowSep}' +// |STORED AS TEXTFILE +// |LOCATION '${tablePath}'""".stripMargin +// sqlContext.sql(sql) +// } catch { +// case e: Throwable => throw e +// } +// } +// +// def available(): Boolean = { +// true +// } +// +// private def encode(data: Map[String, Any], ms: Long): Option[List[Any]] = { +// try { +// Some(schema.map { field => +// val (name, _) = field +// name match { +// case TimeStampColumn => ms +// case PayloadColumn => JsonUtil.toJson(data) +// case _ => null +// } +// }) +// } catch { +// case _ => None +// } +// } +// +// private def decode(data: List[Any], updateTimeStamp: Boolean): Option[Map[String, Any]] = { +// val dataMap = schemaName.zip(data).toMap +// dataMap.get(PayloadColumn) match { +// case Some(v: String) => { +// try { +// val map = JsonUtil.toAnyMap(v) +// val resMap = if (updateTimeStamp) { +// dataMap.get(TimeStampColumn) match { +// case Some(t) => map + (TimeStampColumn -> t) +// case _ => map +// } +// } else map +// Some(resMap) +// } catch { +// case _ => None +// } +// } +// case _ => None +// } +// } +// +// def saveData(rdd: RDD[Map[String, Any]], ms: Long): Unit = { +// val newCacheLocked = newCacheLock.lock(-1, TimeUnit.SECONDS) +// if (newCacheLocked) { +// try { +// val ptns = getPartition(ms) +// val ptnsPath = genPartitionHdfsPath(ptns) +// val dirPath = s"${tablePath}/${ptnsPath}" +// val fileName = s"${ms}" +// val filePath = HdfsUtil.getHdfsFilePath(dirPath, fileName) +// +// // encode data +// val dataRdd: RDD[List[Any]] = rdd.flatMap(encode(_, ms)) +// +// // save data +// val recordRdd: RDD[String] = dataRdd.map { dt => +// dt.map(_.toString).mkString(fieldSep) +// } +// +// val dumped = if (!recordRdd.isEmpty) { +// HdfsFileDumpUtil.dump(filePath, recordRdd, rowSepLiteral) +// } else false +// +// // add partition +// if (dumped) { +// val sql = addPartitionSql(concreteTableName, ptns) +// sqlContext.sql(sql) +// } +// +// // submit ms +// submitCacheTime(ms) +// submitReadyTime(ms) +// } catch { +// case e: Throwable => error(s"save data error: ${e.getMessage}") +// } finally { +// newCacheLock.unlock() +// } +// } +// } +// +// def readData(): Try[RDD[Map[String, Any]]] = Try { +// val timeRange = TimeInfoCache.getTimeRange +// submitLastProcTime(timeRange._2) +// +// val reviseTimeRange = (timeRange._1 + deltaTimeRange._1, timeRange._2 + deltaTimeRange._2) +// submitCleanTime(reviseTimeRange._1) +// +// // read directly through partition info +// val partitionRange = getPartitionRange(reviseTimeRange._1, reviseTimeRange._2) +// val sql = selectSql(concreteTableName, partitionRange) +// val df = sqlContext.sql(sql) +// +// // decode data +// df.flatMap { row => +// val dt = schemaName.map { sn => +// row.getAs[Any](sn) +// } +// decode(dt, true) +// } +// } +// +// override def cleanOldData(): Unit = { +// val oldCacheLocked = oldCacheLock.lock(-1, TimeUnit.SECONDS) +// if (oldCacheLocked) { +// try { +// val cleanTime = readCleanTime() +// cleanTime match { +// case Some(ct) => { +// // drop partition +// val bound = getPartition(ct) +// val sql = dropPartitionSql(concreteTableName, bound) +// sqlContext.sql(sql) +// } +// case _ => { +// // do nothing +// } +// } +// } catch { +// case e: Throwable => error(s"clean old data error: ${e.getMessage}") +// } finally { +// oldCacheLock.unlock() +// } +// } +// } +// +// override def updateOldData(t: Long, oldData: Iterable[Map[String, Any]]): Unit = { +// // parallel process different time groups, lock is unnecessary +// val ptns = getPartition(t) +// val ptnsPath = genPartitionHdfsPath(ptns) +// val dirPath = s"${tablePath}/${ptnsPath}" +// val fileName = s"${t}" +// val filePath = HdfsUtil.getHdfsFilePath(dirPath, fileName) +// +// try { +// // remove out time old data +// HdfsFileDumpUtil.remove(dirPath, fileName, true) +// +// // save updated old data +// if (oldData.size > 0) { +// val recordDatas = oldData.flatMap { dt => +// encode(dt, t) +// } +// val records: Iterable[String] = recordDatas.map { dt => +// dt.map(_.toString).mkString(fieldSep) +// } +// val dumped = HdfsFileDumpUtil.dump(filePath, records, rowSepLiteral) +// } +// } catch { +// case e: Throwable => error(s"update old data error: ${e.getMessage}") +// } +// } +// +// override protected def genCleanTime(ms: Long): Long = { +// val minPartition = partition.last +// val t1 = TimeUtil.timeToUnit(ms, minPartition._3) +// val t2 = TimeUtil.timeFromUnit(t1, minPartition._3) +// t2 +// } +// +// private def getPartition(ms: Long): List[(String, Any)] = { +// partition.map { p => +// val (name, _, unit) = p +// val t = TimeUtil.timeToUnit(ms, unit) +// (name, t) +// } +// } +// private def getPartitionRange(ms1: Long, ms2: Long): List[(String, (Any, Any))] = { +// partition.map { p => +// val (name, _, unit) = p +// val t1 = TimeUtil.timeToUnit(ms1, unit) +// val t2 = TimeUtil.timeToUnit(ms2, unit) +// (name, (t1, t2)) +// } +// } +// +// private def genPartitionHdfsPath(partition: List[(String, Any)]): String = { +// partition.map(prtn => s"${prtn._1}=${prtn._2}").mkString("/") +// } +// private def addPartitionSql(tbn: String, partition: List[(String, Any)]): String = { +// val partitionSql = partition.map(ptn => (s"`${ptn._1}` = ${ptn._2}")).mkString(", ") +// val sql = s"""ALTER TABLE ${tbn} ADD IF NOT EXISTS PARTITION (${partitionSql})""" +// sql +// } +// private def selectSql(tbn: String, partitionRange: List[(String, (Any, Any))]): String = { +// val clause = partitionRange.map { pr => +// val (name, (r1, r2)) = pr +// s"""`${name}` BETWEEN '${r1}' and '${r2}'""" +// }.mkString(" AND ") +// val whereClause = if (clause.nonEmpty) s"WHERE ${clause}" else "" +// val sql = s"""SELECT * FROM ${tbn} ${whereClause}""" +// sql +// } +// private def dropPartitionSql(tbn: String, partition: List[(String, Any)]): String = { +// val partitionSql = partition.map(ptn => (s"PARTITION ( `${ptn._1}` < '${ptn._2}' ) ")).mkString(", ") +// val sql = s"""ALTER TABLE ${tbn} DROP ${partitionSql}""" +// println(sql) +// sql +// } +// +// private def tableExistsSql(): String = { +// s"tableName LIKE '${tableName}'" +// } +// +//} diff --git a/measure/src/main/scala/org/apache/griffin/measure/data/connector/cache/TextCacheDataConnector.scala b/measure/src/main/scala/org/apache/griffin/measure/data/connector/cache/TextCacheDataConnector.scala new file mode 100644 index 000000000..0daf2d986 --- /dev/null +++ b/measure/src/main/scala/org/apache/griffin/measure/data/connector/cache/TextCacheDataConnector.scala @@ -0,0 +1,311 @@ +///* +//Licensed to the Apache Software Foundation (ASF) under one +//or more contributor license agreements. See the NOTICE file +//distributed with this work for additional information +//regarding copyright ownership. The ASF licenses this file +//to you under the Apache License, Version 2.0 (the +//"License"); you may not use this file except in compliance +//with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +//Unless required by applicable law or agreed to in writing, +//software distributed under the License is distributed on an +//"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +//KIND, either express or implied. See the License for the +//specific language governing permissions and limitations +//under the License. +//*/ +//package org.apache.griffin.measure.data.connector.cache +// +//import java.util.concurrent.TimeUnit +// +//import org.apache.griffin.measure.cache.info.{InfoCacheInstance, TimeInfoCache} +//import org.apache.griffin.measure.config.params.user.DataCacheParam +//import org.apache.griffin.measure.result.TimeStampInfo +//import org.apache.griffin.measure.utils.{HdfsFileDumpUtil, HdfsUtil, JsonUtil, TimeUtil} +//import org.apache.spark.rdd.RDD +//import org.apache.spark.sql.SQLContext +// +//import scala.util.Try +// +//case class TextCacheDataConnector(sqlContext: SQLContext, dataCacheParam: DataCacheParam +// ) extends CacheDataConnector { +// +// val config = dataCacheParam.config +// val InfoPath = "info.path" +// val cacheInfoPath: String = config.getOrElse(InfoPath, defCacheInfoPath).toString +// +// val newCacheLock = InfoCacheInstance.genLock(s"${cacheInfoPath}.new") +// val oldCacheLock = InfoCacheInstance.genLock(s"${cacheInfoPath}.old") +// +// val timeRangeParam: List[String] = if (dataCacheParam.timeRange != null) dataCacheParam.timeRange else Nil +// val deltaTimeRange: (Long, Long) = (timeRangeParam ::: List("0", "0")) match { +// case s :: e :: _ => { +// val ns = TimeUtil.milliseconds(s) match { +// case Some(n) if (n < 0) => n +// case _ => 0 +// } +// val ne = TimeUtil.milliseconds(e) match { +// case Some(n) if (n < 0) => n +// case _ => 0 +// } +// (ns, ne) +// } +// case _ => (0, 0) +// } +// +// val FilePath = "file.path" +// val filePath: String = config.get(FilePath) match { +// case Some(s: String) => s +// case _ => throw new Exception("invalid file.path!") +// } +// +// val ReadyTimeInterval = "ready.time.interval" +// val ReadyTimeDelay = "ready.time.delay" +// val readyTimeInterval: Long = TimeUtil.milliseconds(config.getOrElse(ReadyTimeInterval, "1m").toString).getOrElse(60000L) +// val readyTimeDelay: Long = TimeUtil.milliseconds(config.getOrElse(ReadyTimeDelay, "1m").toString).getOrElse(60000L) +// +//// val TimeStampColumn: String = TimeStampInfo.key +//// val PayloadColumn: String = "payload" +// +// // cache schema: Long, String +//// val fields = List[StructField]( +//// StructField(TimeStampColumn, LongType), +//// StructField(PayloadColumn, StringType) +//// ) +//// val schema = StructType(fields) +// +// // case class CacheData(time: Long, payload: String) { +// // def getTime(): Long = time +// // def getPayload(): String = payload +// // } +// +// private val rowSepLiteral = "\n" +// +// val partitionUnits: List[String] = List("hour", "min") +// +// override def init(): Unit = { +// // do nothing +// } +// +// def available(): Boolean = { +// true +// } +// +// private def encode(data: Map[String, Any], ms: Long): Option[String] = { +// try { +// val map = data + (TimeStampInfo.key -> ms) +// Some(JsonUtil.toJson(map)) +// } catch { +// case _: Throwable => None +// } +// } +// +// private def decode(data: String): Option[Map[String, Any]] = { +// try { +// Some(JsonUtil.toAnyMap(data)) +// } catch { +// case _: Throwable => None +// } +// } +// +// def saveData(rdd: RDD[Map[String, Any]], ms: Long): Unit = { +// val newCacheLocked = newCacheLock.lock(-1, TimeUnit.SECONDS) +// if (newCacheLocked) { +// try { +// val ptns = getPartition(ms) +// val ptnsPath = genPartitionHdfsPath(ptns) +// val dirPath = s"${filePath}/${ptnsPath}" +// val dataFileName = s"${ms}" +// val dataFilePath = HdfsUtil.getHdfsFilePath(dirPath, dataFileName) +// +// // encode data +// val dataRdd: RDD[String] = rdd.flatMap(encode(_, ms)) +// +// // save data +// val dumped = if (!dataRdd.isEmpty) { +// HdfsFileDumpUtil.dump(dataFilePath, dataRdd, rowSepLiteral) +// } else false +// +// // submit ms +// submitCacheTime(ms) +// submitReadyTime(ms) +// } catch { +// case e: Throwable => error(s"save data error: ${e.getMessage}") +// } finally { +// newCacheLock.unlock() +// } +// } +// } +// +// def readData(): Try[RDD[Map[String, Any]]] = Try { +// val timeRange = TimeInfoCache.getTimeRange +// submitLastProcTime(timeRange._2) +// +// val reviseTimeRange = (timeRange._1 + deltaTimeRange._1, timeRange._2 + deltaTimeRange._2) +// submitCleanTime(reviseTimeRange._1) +// +// // read directly through partition info +// val partitionRanges = getPartitionRange(reviseTimeRange._1, reviseTimeRange._2) +// println(s"read time ranges: ${reviseTimeRange}") +// println(s"read partition ranges: ${partitionRanges}") +// +// // list partition paths +// val partitionPaths = listPathsBetweenRanges(filePath :: Nil, partitionRanges) +// +// if (partitionPaths.isEmpty) { +// sqlContext.sparkContext.emptyRDD[Map[String, Any]] +// } else { +// val filePaths = partitionPaths.mkString(",") +// val rdd = sqlContext.sparkContext.textFile(filePaths) +// +// // decode data +// rdd.flatMap { row => +// decode(row) +// } +// } +// } +// +// override def cleanOldData(): Unit = { +// val oldCacheLocked = oldCacheLock.lock(-1, TimeUnit.SECONDS) +// if (oldCacheLocked) { +// try { +// val cleanTime = readCleanTime() +// cleanTime match { +// case Some(ct) => { +// // drop partitions +// val bounds = getPartition(ct) +// +// // list partition paths +// val earlierPaths = listPathsEarlierThanBounds(filePath :: Nil, bounds) +// +// // delete out time data path +// earlierPaths.foreach { path => +// println(s"delete hdfs path: ${path}") +// HdfsUtil.deleteHdfsPath(path) +// } +// } +// case _ => { +// // do nothing +// } +// } +// } catch { +// case e: Throwable => error(s"clean old data error: ${e.getMessage}") +// } finally { +// oldCacheLock.unlock() +// } +// } +// } +// +// override def updateOldData(t: Long, oldData: Iterable[Map[String, Any]]): Unit = { +// // parallel process different time groups, lock is unnecessary +// val ptns = getPartition(t) +// val ptnsPath = genPartitionHdfsPath(ptns) +// val dirPath = s"${filePath}/${ptnsPath}" +// val dataFileName = s"${t}" +// val dataFilePath = HdfsUtil.getHdfsFilePath(dirPath, dataFileName) +// +// try { +// // remove out time old data +// HdfsFileDumpUtil.remove(dirPath, dataFileName, true) +// +// // save updated old data +// if (oldData.size > 0) { +// val recordDatas = oldData.flatMap { dt => +// encode(dt, t) +// } +// val dumped = HdfsFileDumpUtil.dump(dataFilePath, recordDatas, rowSepLiteral) +// } +// } catch { +// case e: Throwable => error(s"update old data error: ${e.getMessage}") +// } +// } +// +// override protected def genCleanTime(ms: Long): Long = { +// val minPartitionUnit = partitionUnits.last +// val t1 = TimeUtil.timeToUnit(ms, minPartitionUnit) +// val t2 = TimeUtil.timeFromUnit(t1, minPartitionUnit) +// t2 +// } +// +// private def getPartition(ms: Long): List[Long] = { +// partitionUnits.map { unit => +// TimeUtil.timeToUnit(ms, unit) +// } +// } +// private def getPartitionRange(ms1: Long, ms2: Long): List[(Long, Long)] = { +// partitionUnits.map { unit => +// val t1 = TimeUtil.timeToUnit(ms1, unit) +// val t2 = TimeUtil.timeToUnit(ms2, unit) +// (t1, t2) +// } +// } +// +// private def genPartitionHdfsPath(partition: List[Long]): String = { +// partition.map(prtn => s"${prtn}").mkString("/") +// } +// +// private def str2Long(str: String): Option[Long] = { +// try { +// Some(str.toLong) +// } catch { +// case e: Throwable => None +// } +// } +// +// // here the range means [min, max], but the best range should be (min, max] +// private def listPathsBetweenRanges(paths: List[String], +// partitionRanges: List[(Long, Long)] +// ): List[String] = { +// partitionRanges match { +// case Nil => paths +// case head :: tail => { +// val (lb, ub) = head +// val curPaths = paths.flatMap { path => +// val names = HdfsUtil.listSubPaths(path, "dir").toList +// names.filter { name => +// str2Long(name) match { +// case Some(t) => (t >= lb) && (t <= ub) +// case _ => false +// } +// }.map(HdfsUtil.getHdfsFilePath(path, _)) +// } +// listPathsBetweenRanges(curPaths, tail) +// } +// } +// } +// +// private def listPathsEarlierThanBounds(paths: List[String], bounds: List[Long] +// ): List[String] = { +// bounds match { +// case Nil => paths +// case head :: tail => { +// val earlierPaths = paths.flatMap { path => +// val names = HdfsUtil.listSubPaths(path, "dir").toList +// names.filter { name => +// str2Long(name) match { +// case Some(t) => (t < head) +// case _ => false +// } +// }.map(HdfsUtil.getHdfsFilePath(path, _)) +// } +// val equalPaths = paths.flatMap { path => +// val names = HdfsUtil.listSubPaths(path, "dir").toList +// names.filter { name => +// str2Long(name) match { +// case Some(t) => (t == head) +// case _ => false +// } +// }.map(HdfsUtil.getHdfsFilePath(path, _)) +// } +// +// tail match { +// case Nil => earlierPaths +// case _ => earlierPaths ::: listPathsEarlierThanBounds(equalPaths, tail) +// } +// } +// } +// } +// +//} diff --git a/measure/src/main/scala/org/apache/griffin/measure/data/connector/direct/AvroDirectDataConnector.scala b/measure/src/main/scala/org/apache/griffin/measure/data/connector/direct/AvroDirectDataConnector.scala new file mode 100644 index 000000000..476401d32 --- /dev/null +++ b/measure/src/main/scala/org/apache/griffin/measure/data/connector/direct/AvroDirectDataConnector.scala @@ -0,0 +1,105 @@ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ +package org.apache.griffin.measure.data.connector.direct + +import org.apache.griffin.measure.result._ +import org.apache.griffin.measure.rule.{ExprValueUtil, RuleExprs} +import org.apache.griffin.measure.utils.HdfsUtil +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.{DataFrame, SQLContext} + +import scala.util.Try + +// data connector for avro file +case class AvroDirectDataConnector(sqlContext: SQLContext, config: Map[String, Any] + ) extends DirectDataConnector { + + val FilePath = "file.path" + val FileName = "file.name" + + val filePath = config.getOrElse(FilePath, "").toString + val fileName = config.getOrElse(FileName, "").toString + + val concreteFileFullPath = if (pathPrefix) s"${filePath}${fileName}" else fileName + + private def pathPrefix(): Boolean = { + filePath.nonEmpty + } + + private def fileExist(): Boolean = { + HdfsUtil.existPath(concreteFileFullPath) + } + + def data(): Option[DataFrame] = { + try { + val df = sqlContext.read.format("com.databricks.spark.avro").load(concreteFileFullPath) + Some(df) + } catch { + case e: Throwable => { + error(s"load avro file ${concreteFileFullPath} fails") + None + } + } + } + +// def available(): Boolean = { +// (!concreteFileFullPath.isEmpty) && fileExist +// } + +// def init(): Unit = {} + +// def metaData(): Try[Iterable[(String, String)]] = { +// Try { +// val st = sqlContext.read.format("com.databricks.spark.avro").load(concreteFileFullPath).schema +// st.fields.map(f => (f.name, f.dataType.typeName)) +// } +// } + +// def data(): Try[RDD[(Product, (Map[String, Any], Map[String, Any]))]] = { +// Try { +// loadDataFile.flatMap { row => +// // generate cache data +// val cacheExprValueMaps = ExprValueUtil.genExprValueMaps(Some(row), ruleExprs.cacheExprs, constFinalExprValueMap) +// val finalExprValueMaps = ExprValueUtil.updateExprValueMaps(ruleExprs.finalCacheExprs, cacheExprValueMaps) +// +// // data info +// val dataInfoMap: Map[String, Any] = DataInfo.cacheInfoList.map { info => +// try { +// (info.key -> row.getAs[info.T](info.key)) +// } catch { +// case e: Throwable => info.defWrap +// } +// }.toMap +// +// finalExprValueMaps.flatMap { finalExprValueMap => +// val groupbyData: Seq[AnyRef] = ruleExprs.groupbyExprs.flatMap { expr => +// expr.calculate(finalExprValueMap) match { +// case Some(v) => Some(v.asInstanceOf[AnyRef]) +// case _ => None +// } +// } +// val key = toTuple(groupbyData) +// +// Some((key, (finalExprValueMap, dataInfoMap))) +// } +// } +// } +// } + +} diff --git a/measure/src/main/scala/org/apache/griffin/measure/data/connector/direct/DirectDataConnector.scala b/measure/src/main/scala/org/apache/griffin/measure/data/connector/direct/DirectDataConnector.scala new file mode 100644 index 000000000..93e168e78 --- /dev/null +++ b/measure/src/main/scala/org/apache/griffin/measure/data/connector/direct/DirectDataConnector.scala @@ -0,0 +1,35 @@ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ +package org.apache.griffin.measure.data.connector.direct + +import org.apache.griffin.measure.data.connector.DataConnector +//import org.apache.griffin.measure.data.connector.cache.DataUpdatable +import org.apache.spark.sql.DataFrame +import org.apache.spark.sql.types.StructType + +import scala.util.{Failure, Success, Try} + + +trait DirectDataConnector extends DataConnector { + +// def metaData(): Option[StructType] + + def data(): Option[DataFrame] + +} diff --git a/measure/src/main/scala/org/apache/griffin/measure/data/connector/direct/HiveDirectDataConnector.scala b/measure/src/main/scala/org/apache/griffin/measure/data/connector/direct/HiveDirectDataConnector.scala new file mode 100644 index 000000000..a2fed3ef6 --- /dev/null +++ b/measure/src/main/scala/org/apache/griffin/measure/data/connector/direct/HiveDirectDataConnector.scala @@ -0,0 +1,142 @@ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ +package org.apache.griffin.measure.data.connector.direct + +import org.apache.griffin.measure.result._ +import org.apache.griffin.measure.rule.{ExprValueUtil, RuleExprs} +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.hive.HiveContext +import org.apache.spark.sql.{DataFrame, SQLContext} + +import scala.util.{Success, Try} + +// data connector for hive +case class HiveDirectDataConnector(sqlContext: SQLContext, config: Map[String, Any] + ) extends DirectDataConnector { + + if (!sqlContext.isInstanceOf[HiveContext]) { + throw new Exception("hive context not prepared!") + } + + val Database = "database" + val TableName = "table.name" + val Partitions = "partitions" + + val database = config.getOrElse(Database, "default").toString + val tableName = config.getOrElse(TableName, "").toString + val partitionsString = config.getOrElse(Partitions, "").toString + + val concreteTableName = s"${database}.${tableName}" + val partitions = partitionsString.split(";").map(s => s.split(",").map(_.trim)) + + def data(): Option[DataFrame] = { + try { + val df = sqlContext.sql(dataSql) + Some(df) + } catch { + case e: Throwable => { + error(s"load hive table ${concreteTableName} fails") + None + } + } + } + +// def available(): Boolean = { +// (!tableName.isEmpty) && { +// Try { +// if (dbPrefix) { +// sqlContext.tables(database).filter(tableExistsSql).collect.size +// } else { +// sqlContext.tables().filter(tableExistsSql).collect.size +// } +// } match { +// case Success(s) => s > 0 +// case _ => false +// } +// } +// } + +// def init(): Unit = {} + +// def metaData(): Try[Iterable[(String, String)]] = { +// Try { +// val originRows = sqlContext.sql(metaDataSql).map(r => (r.getString(0), r.getString(1))).collect +// val partitionPos: Int = originRows.indexWhere(pair => pair._1.startsWith("# ")) +// if (partitionPos < 0) originRows +// else originRows.take(partitionPos) +// } +// } + +// def data(): Try[RDD[(Product, (Map[String, Any], Map[String, Any]))]] = { +// Try { +// sqlContext.sql(dataSql).flatMap { row => +// // generate cache data +// val cacheExprValueMaps = ExprValueUtil.genExprValueMaps(Some(row), ruleExprs.cacheExprs, constFinalExprValueMap) +// val finalExprValueMaps = ExprValueUtil.updateExprValueMaps(ruleExprs.finalCacheExprs, cacheExprValueMaps) +// +// // data info +// val dataInfoMap: Map[String, Any] = DataInfo.cacheInfoList.map { info => +// try { +// (info.key -> row.getAs[info.T](info.key)) +// } catch { +// case e: Throwable => info.defWrap +// } +// }.toMap +// +// finalExprValueMaps.flatMap { finalExprValueMap => +// val groupbyData: Seq[AnyRef] = ruleExprs.groupbyExprs.flatMap { expr => +// expr.calculate(finalExprValueMap) match { +// case Some(v) => Some(v.asInstanceOf[AnyRef]) +// case _ => None +// } +// } +// val key = toTuple(groupbyData) +// +// Some((key, (finalExprValueMap, dataInfoMap))) +// } +// } +// } +// } + + private def tableExistsSql(): String = { +// s"SHOW TABLES LIKE '${concreteTableName}'" // this is hive sql, but not work for spark sql + s"tableName LIKE '${tableName}'" + } + + private def metaDataSql(): String = { + s"DESCRIBE ${concreteTableName}" + } + + private def dataSql(): String = { + val clauses = partitions.map { prtn => + val cls = prtn.mkString(" AND ") + if (cls.isEmpty) s"SELECT * FROM ${concreteTableName}" + else s"SELECT * FROM ${concreteTableName} WHERE ${cls}" + } + clauses.mkString(" UNION ALL ") + } + +// private def toTuple[A <: AnyRef](as: Seq[A]): Product = { +// if (as.size > 0) { +// val tupleClass = Class.forName("scala.Tuple" + as.size) +// tupleClass.getConstructors.apply(0).newInstance(as: _*).asInstanceOf[Product] +// } else None +// } + +} diff --git a/measure/src/main/scala/org/apache/griffin/measure/data/connector/direct/KafkaCacheDirectDataConnector.scala b/measure/src/main/scala/org/apache/griffin/measure/data/connector/direct/KafkaCacheDirectDataConnector.scala new file mode 100644 index 000000000..70ddcde60 --- /dev/null +++ b/measure/src/main/scala/org/apache/griffin/measure/data/connector/direct/KafkaCacheDirectDataConnector.scala @@ -0,0 +1,125 @@ +///* +//Licensed to the Apache Software Foundation (ASF) under one +//or more contributor license agreements. See the NOTICE file +//distributed with this work for additional information +//regarding copyright ownership. The ASF licenses this file +//to you under the Apache License, Version 2.0 (the +//"License"); you may not use this file except in compliance +//with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +//Unless required by applicable law or agreed to in writing, +//software distributed under the License is distributed on an +//"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +//KIND, either express or implied. See the License for the +//specific language governing permissions and limitations +//under the License. +//*/ +//package org.apache.griffin.measure.data.connector.direct +// +//import org.apache.griffin.measure.config.params.user.DataConnectorParam +//import org.apache.griffin.measure.data.connector.DataConnectorFactory +//import org.apache.griffin.measure.data.connector.cache.CacheDataConnector +//import org.apache.griffin.measure.data.connector.streaming.StreamingDataConnector +//import org.apache.griffin.measure.result._ +//import org.apache.griffin.measure.rule._ +//import org.apache.spark.rdd.RDD +//import org.apache.spark.sql.SQLContext +//import org.apache.spark.streaming.StreamingContext +// +//import scala.util.{Failure, Success, Try} +// +//case class KafkaCacheDirectDataConnector(@transient streamingDataConnectorTry: Try[StreamingDataConnector], +// cacheDataConnectorTry: Try[CacheDataConnector], +// dataConnectorParam: DataConnectorParam, +// ruleExprs: RuleExprs, +// constFinalExprValueMap: Map[String, Any] +// ) extends StreamingCacheDirectDataConnector { +// +// val cacheDataConnector: CacheDataConnector = cacheDataConnectorTry match { +// case Success(cntr) => cntr +// case Failure(ex) => throw ex +// } +// @transient val streamingDataConnector: StreamingDataConnector = streamingDataConnectorTry match { +// case Success(cntr) => cntr +// case Failure(ex) => throw ex +// } +// +// protected def transform(rdd: RDD[(streamingDataConnector.K, streamingDataConnector.V)], +// ms: Long +// ): RDD[Map[String, Any]] = { +// val dataInfoMap = DataInfo.cacheInfoList.map(_.defWrap).toMap + TimeStampInfo.wrap(ms) +// +// rdd.flatMap { kv => +// val msg = kv._2 +// +// val cacheExprValueMaps = ExprValueUtil.genExprValueMaps(Some(msg), ruleExprs.cacheExprs, constFinalExprValueMap) +// val finalExprValueMaps = ExprValueUtil.updateExprValueMaps(ruleExprs.finalCacheExprs, cacheExprValueMaps) +// +// finalExprValueMaps.map { vm => +// vm ++ dataInfoMap +// } +// } +// } +// +// def metaData(): Try[Iterable[(String, String)]] = Try { +// Map.empty[String, String] +// } +// +// def data(): Try[RDD[(Product, (Map[String, Any], Map[String, Any]))]] = Try { +// cacheDataConnector.readData match { +// case Success(rdd) => { +// rdd.flatMap { row => +// val finalExprValueMap = ruleExprs.finalCacheExprs.flatMap { expr => +// row.get(expr._id).flatMap { d => +// Some((expr._id, d)) +// } +// }.toMap +// +// val dataInfoMap: Map[String, Any] = DataInfo.cacheInfoList.map { info => +// row.get(info.key) match { +// case Some(d) => (info.key -> d) +// case _ => info.defWrap +// } +// }.toMap +// +// val groupbyData: Seq[AnyRef] = ruleExprs.groupbyExprs.flatMap { expr => +// expr.calculate(finalExprValueMap) match { +// case Some(v) => Some(v.asInstanceOf[AnyRef]) +// case _ => None +// } +// } +// val key = toTuple(groupbyData) +// +// Some((key, (finalExprValueMap, dataInfoMap))) +// } +// } +// case Failure(ex) => throw ex +// } +// } +// +// override def cleanOldData(): Unit = { +// cacheDataConnector.cleanOldData +// } +// +// override def updateOldData(t: Long, oldData: Iterable[Map[String, Any]]): Unit = { +// if (dataConnectorParam.getMatchOnce) { +// cacheDataConnector.updateOldData(t, oldData) +// } +// } +// +// override def updateAllOldData(oldRdd: RDD[Map[String, Any]]): Unit = { +// if (dataConnectorParam.getMatchOnce) { +// cacheDataConnector.updateAllOldData(oldRdd) +// } +// } +// +// private def toTuple[A <: AnyRef](as: Seq[A]): Product = { +// if (as.size > 0) { +// val tupleClass = Class.forName("scala.Tuple" + as.size) +// tupleClass.getConstructors.apply(0).newInstance(as: _*).asInstanceOf[Product] +// } else None +// } +// +//} diff --git a/measure/src/main/scala/org/apache/griffin/measure/data/connector/direct/StreamingCacheDirectDataConnector.scala b/measure/src/main/scala/org/apache/griffin/measure/data/connector/direct/StreamingCacheDirectDataConnector.scala new file mode 100644 index 000000000..dddf4309b --- /dev/null +++ b/measure/src/main/scala/org/apache/griffin/measure/data/connector/direct/StreamingCacheDirectDataConnector.scala @@ -0,0 +1,60 @@ +///* +//Licensed to the Apache Software Foundation (ASF) under one +//or more contributor license agreements. See the NOTICE file +//distributed with this work for additional information +//regarding copyright ownership. The ASF licenses this file +//to you under the Apache License, Version 2.0 (the +//"License"); you may not use this file except in compliance +//with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +//Unless required by applicable law or agreed to in writing, +//software distributed under the License is distributed on an +//"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +//KIND, either express or implied. See the License for the +//specific language governing permissions and limitations +//under the License. +//*/ +//package org.apache.griffin.measure.data.connector.direct +// +//import org.apache.griffin.measure.data.connector.cache.CacheDataConnector +//import org.apache.griffin.measure.data.connector.streaming.StreamingDataConnector +//import org.apache.griffin.measure.result.{DataInfo, TimeStampInfo} +//import org.apache.griffin.measure.rule.ExprValueUtil +//import org.apache.spark.rdd.RDD +// +//import scala.util.{Failure, Success} +// +//trait StreamingCacheDirectDataConnector extends DirectDataConnector { +// +// val cacheDataConnector: CacheDataConnector +// @transient val streamingDataConnector: StreamingDataConnector +// +// def available(): Boolean = { +// cacheDataConnector.available && streamingDataConnector.available +// } +// +// def init(): Unit = { +// cacheDataConnector.init +// +// val ds = streamingDataConnector.stream match { +// case Success(dstream) => dstream +// case Failure(ex) => throw ex +// } +// +// ds.foreachRDD((rdd, time) => { +// val ms = time.milliseconds +// +// val valueMapRdd = transform(rdd, ms) +// +// // save data frame +// cacheDataConnector.saveData(valueMapRdd, ms) +// }) +// } +// +// protected def transform(rdd: RDD[(streamingDataConnector.K, streamingDataConnector.V)], +// ms: Long +// ): RDD[Map[String, Any]] +// +//} diff --git a/measure/src/main/scala/org/apache/griffin/measure/data/connector/streaming/KafkaStreamingDataConnector.scala b/measure/src/main/scala/org/apache/griffin/measure/data/connector/streaming/KafkaStreamingDataConnector.scala new file mode 100644 index 000000000..0abb826ef --- /dev/null +++ b/measure/src/main/scala/org/apache/griffin/measure/data/connector/streaming/KafkaStreamingDataConnector.scala @@ -0,0 +1,58 @@ +///* +//Licensed to the Apache Software Foundation (ASF) under one +//or more contributor license agreements. See the NOTICE file +//distributed with this work for additional information +//regarding copyright ownership. The ASF licenses this file +//to you under the Apache License, Version 2.0 (the +//"License"); you may not use this file except in compliance +//with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +//Unless required by applicable law or agreed to in writing, +//software distributed under the License is distributed on an +//"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +//KIND, either express or implied. See the License for the +//specific language governing permissions and limitations +//under the License. +//*/ +//package org.apache.griffin.measure.data.connector.streaming +// +//import kafka.serializer.Decoder +//import org.apache.griffin.measure.data.connector.cache.{CacheDataConnector, DataCacheable} +//import org.apache.griffin.measure.result.{DataInfo, TimeStampInfo} +//import org.apache.griffin.measure.rule.{ExprValueUtil, RuleExprs} +//import org.apache.spark.rdd.RDD +//import org.apache.spark.streaming.StreamingContext +//import org.apache.spark.streaming.dstream.InputDStream +// +//import scala.util.{Failure, Success, Try} +// +//abstract class KafkaStreamingDataConnector(@transient ssc: StreamingContext, +// config: Map[String, Any] +// ) extends StreamingDataConnector { +// type KD <: Decoder[K] +// type VD <: Decoder[V] +// +// val KafkaConfig = "kafka.config" +// val Topics = "topics" +// +// val kafkaConfig = config.get(KafkaConfig) match { +// case Some(map: Map[String, Any]) => map.mapValues(_.toString).map(identity) +// case _ => Map[String, String]() +// } +// val topics = config.getOrElse(Topics, "").toString +// +// def available(): Boolean = { +// true +// } +// +// def init(): Unit = {} +// +// def stream(): Try[InputDStream[(K, V)]] = Try { +// val topicSet = topics.split(",").toSet +// createDStream(topicSet) +// } +// +// protected def createDStream(topicSet: Set[String]): InputDStream[(K, V)] +//} \ No newline at end of file diff --git a/measure/src/main/scala/org/apache/griffin/measure/data/connector/streaming/StreamingDataConnector.scala b/measure/src/main/scala/org/apache/griffin/measure/data/connector/streaming/StreamingDataConnector.scala new file mode 100644 index 000000000..3267ec77b --- /dev/null +++ b/measure/src/main/scala/org/apache/griffin/measure/data/connector/streaming/StreamingDataConnector.scala @@ -0,0 +1,34 @@ +///* +//Licensed to the Apache Software Foundation (ASF) under one +//or more contributor license agreements. See the NOTICE file +//distributed with this work for additional information +//regarding copyright ownership. The ASF licenses this file +//to you under the Apache License, Version 2.0 (the +//"License"); you may not use this file except in compliance +//with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +//Unless required by applicable law or agreed to in writing, +//software distributed under the License is distributed on an +//"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +//KIND, either express or implied. See the License for the +//specific language governing permissions and limitations +//under the License. +//*/ +//package org.apache.griffin.measure.data.connector.streaming +// +//import org.apache.griffin.measure.data.connector.DataConnector +//import org.apache.spark.streaming.dstream.InputDStream +// +//import scala.util.Try +// +// +//trait StreamingDataConnector extends DataConnector { +// +// type K +// type V +// +// def stream(): Try[InputDStream[(K, V)]] +// +//} diff --git a/measure/src/main/scala/org/apache/griffin/measure/data/source/DataSource.scala b/measure/src/main/scala/org/apache/griffin/measure/data/source/DataSource.scala new file mode 100644 index 000000000..0b31a8ed4 --- /dev/null +++ b/measure/src/main/scala/org/apache/griffin/measure/data/source/DataSource.scala @@ -0,0 +1,26 @@ +package org.apache.griffin.measure.data.source + +import org.apache.griffin.measure.data.connector.direct._ +import org.apache.griffin.measure.log.Loggable +import org.apache.spark.sql.{DataFrame, SQLContext} + +case class DataSource(name: String, dataConnectors: Seq[DirectDataConnector]) extends Loggable with Serializable { + + def init(): Unit = { + data match { + case Some(df) => { + df.registerTempTable(name) + } + case None => { + throw new Exception(s"load data source ${name} fails") + } + } + } + + private def data(): Option[DataFrame] = { + dataConnectors.flatMap { dc => + dc.data + }.reduceOption(_ unionAll _) + } + +} diff --git a/measure/src/main/scala/org/apache/griffin/measure/data/source/DataSourceFactory.scala b/measure/src/main/scala/org/apache/griffin/measure/data/source/DataSourceFactory.scala new file mode 100644 index 000000000..c737a8435 --- /dev/null +++ b/measure/src/main/scala/org/apache/griffin/measure/data/source/DataSourceFactory.scala @@ -0,0 +1,43 @@ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ +package org.apache.griffin.measure.data.source + +import org.apache.griffin.measure.config.params.user.DataSourceParam +import org.apache.griffin.measure.data.connector.DataConnectorFactory +import org.apache.spark.sql.SQLContext +import org.apache.spark.streaming.StreamingContext + +import scala.util.Success + +object DataSourceFactory { + + def genDataSource(sqlContext: SQLContext, ssc: StreamingContext, dataSourceParam: DataSourceParam + ): Option[DataSource] = { + val name = dataSourceParam.name + val connectorParams = dataSourceParam.connectors + val dataConnectors = connectorParams.flatMap { connectorParam => + DataConnectorFactory.getDirectDataConnector(sqlContext, ssc, connectorParam) match { + case Success(connector) => Some(connector) + case _ => None + } + } + Some(DataSource(name, dataConnectors)) + } + +} diff --git a/measure/src/main/scala/org/apache/griffin/measure/persist/HdfsPersist.scala b/measure/src/main/scala/org/apache/griffin/measure/persist/HdfsPersist.scala index 97786c47a..1008e32d4 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/persist/HdfsPersist.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/persist/HdfsPersist.scala @@ -34,8 +34,8 @@ case class HdfsPersist(config: Map[String, Any], metricName: String, timeStamp: val MaxLinesPerFile = "max.lines.per.file" val path = config.getOrElse(Path, "").toString - val maxPersistLines = try { config.getOrElse(MaxPersistLines, -1).toString.toInt } catch { case _ => -1 } - val maxLinesPerFile = try { config.getOrElse(MaxLinesPerFile, 10000).toString.toLong } catch { case _ => 10000 } + val maxPersistLines = try { config.getOrElse(MaxPersistLines, -1).toString.toInt } catch { case _: Throwable => -1 } + val maxLinesPerFile = try { config.getOrElse(MaxLinesPerFile, 10000).toString.toLong } catch { case _: Throwable => 10000 } val separator = "/" @@ -167,16 +167,16 @@ case class HdfsPersist(config: Map[String, Any], metricName: String, timeStamp: def records(recs: RDD[String], tp: String): Unit = { tp match { - case PersistType.MISS => rddRecords(recs, MissRecFile) - case PersistType.MATCH => rddRecords(recs, MatchRecFile) + case PersistDataType.MISS => rddRecords(recs, MissRecFile) + case PersistDataType.MATCH => rddRecords(recs, MatchRecFile) case _ => {} } } def records(recs: Iterable[String], tp: String): Unit = { tp match { - case PersistType.MISS => iterableRecords(recs, MissRecFile) - case PersistType.MATCH => iterableRecords(recs, MatchRecFile) + case PersistDataType.MISS => iterableRecords(recs, MissRecFile) + case PersistDataType.MATCH => iterableRecords(recs, MatchRecFile) case _ => {} } } diff --git a/measure/src/main/scala/org/apache/griffin/measure/persist/LoggerPersist.scala b/measure/src/main/scala/org/apache/griffin/measure/persist/LoggerPersist.scala index 00d41ea28..753aa80ea 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/persist/LoggerPersist.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/persist/LoggerPersist.scala @@ -29,7 +29,7 @@ case class LoggerPersist(config: Map[String, Any], metricName: String, timeStamp val MaxLogLines = "max.log.lines" - val maxLogLines = try { config.getOrElse(MaxLogLines, 100).toString.toInt } catch { case _ => 100 } + val maxLogLines = try { config.getOrElse(MaxLogLines, 100).toString.toInt } catch { case _: Throwable => 100 } def available(): Boolean = true @@ -88,16 +88,16 @@ case class LoggerPersist(config: Map[String, Any], metricName: String, timeStamp def records(recs: RDD[String], tp: String): Unit = { tp match { - case PersistType.MISS => rddRecords(recs) - case PersistType.MATCH => rddRecords(recs) + case PersistDataType.MISS => rddRecords(recs) + case PersistDataType.MATCH => rddRecords(recs) case _ => {} } } def records(recs: Iterable[String], tp: String): Unit = { tp match { - case PersistType.MISS => iterableRecords(recs) - case PersistType.MATCH => iterableRecords(recs) + case PersistDataType.MISS => iterableRecords(recs) + case PersistDataType.MATCH => iterableRecords(recs) case _ => {} } } diff --git a/measure/src/main/scala/org/apache/griffin/measure/persist/Persist.scala b/measure/src/main/scala/org/apache/griffin/measure/persist/Persist.scala index bc16599c6..7db7f51de 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/persist/Persist.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/persist/Persist.scala @@ -46,7 +46,7 @@ trait Persist extends Loggable with Serializable { def log(rt: Long, msg: String): Unit } -object PersistType { +object PersistDataType { final val MISS = "miss" final val MATCH = "match" } \ No newline at end of file diff --git a/measure/src/main/scala/org/apache/griffin/measure/process/BatchDqProcess.scala b/measure/src/main/scala/org/apache/griffin/measure/process/BatchDqProcess.scala new file mode 100644 index 000000000..250f84076 --- /dev/null +++ b/measure/src/main/scala/org/apache/griffin/measure/process/BatchDqProcess.scala @@ -0,0 +1,99 @@ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ +package org.apache.griffin.measure.process + +import java.util.Date + +import org.apache.griffin.measure.config.params._ +import org.apache.griffin.measure.config.params.env._ +import org.apache.griffin.measure.config.params.user._ +import org.apache.griffin.measure.persist.{Persist, PersistFactory} +import org.apache.griffin.measure.process.engine.{DqEngineFactory, SparkSqlEngine} +import org.apache.griffin.measure.rules.adaptor.RuleAdaptorGroup +import org.apache.griffin.measure.utils.JsonUtil +import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.hive.HiveContext +import org.apache.spark.{SparkConf, SparkContext} + +import scala.util.Try + +case class BatchDqProcess(allParam: AllParam) extends DqProcess { + + val envParam: EnvParam = allParam.envParam + val userParam: UserParam = allParam.userParam + + val metricName = userParam.name + val sparkParam = envParam.sparkParam + + var sparkContext: SparkContext = _ + var sqlContext: SQLContext = _ + + def retriable: Boolean = false + + def init: Try[_] = Try { + val conf = new SparkConf().setAppName(metricName) + conf.setAll(sparkParam.config) + sparkContext = new SparkContext(conf) + sparkContext.setLogLevel(sparkParam.logLevel) + sqlContext = new HiveContext(sparkContext) + + // init adaptors + val dataSourceNames = userParam.dataSources.map(_.name) + RuleAdaptorGroup.init(sqlContext, dataSourceNames) + } + + def run: Try[_] = Try { + // start time + val startTime = new Date().getTime() + + // get persists to persist measure result + val persist: Persist = PersistFactory(envParam.persistParams, metricName).getPersists(startTime) + + // persist start id + val applicationId = sparkContext.applicationId + persist.start(applicationId) + + // generate rule steps + val ruleSteps = RuleAdaptorGroup.genConcreteRuleSteps(userParam.evaluateRuleParam) + + // get calculation engine + val engine = DqEngineFactory.genDqEngines(sqlContext, null) + + // init data sources + engine.initDataSources(userParam.dataSources) + + // run rules + engine.runRuleSteps(ruleSteps) + + // persist results + engine.persistResults(ruleSteps, persist) + + // end time + val endTime = new Date().getTime + persist.log(endTime, s"process using time: ${endTime - startTime} ms") + + // finish + persist.finish() + } + + def end: Try[_] = Try { + sparkContext.stop + } + +} diff --git a/measure/src/main/scala/org/apache/griffin/measure/connector/direct/DirectDataConnector.scala b/measure/src/main/scala/org/apache/griffin/measure/process/DqProcess.scala similarity index 66% rename from measure/src/main/scala/org/apache/griffin/measure/connector/direct/DirectDataConnector.scala rename to measure/src/main/scala/org/apache/griffin/measure/process/DqProcess.scala index ac1a79275..50b04a8df 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/connector/direct/DirectDataConnector.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/process/DqProcess.scala @@ -16,19 +16,25 @@ KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License. */ -package org.apache.griffin.measure.connector.direct +package org.apache.griffin.measure.process -import org.apache.griffin.measure.connector.DataConnector -import org.apache.griffin.measure.connector.cache.DataUpdatable -import org.apache.spark.rdd.RDD +import org.apache.griffin.measure.config.params.env._ +import org.apache.griffin.measure.config.params.user._ +import org.apache.griffin.measure.log.Loggable import scala.util.Try +trait DqProcess extends Loggable with Serializable { -trait DirectDataConnector extends DataConnector with DataUpdatable { + val envParam: EnvParam + val userParam: UserParam - def metaData(): Try[Iterable[(String, String)]] + def init: Try[_] - def data(): Try[RDD[(Product, (Map[String, Any], Map[String, Any]))]] + def run: Try[_] + + def end: Try[_] + + def retriable: Boolean } diff --git a/measure/src/main/scala/org/apache/griffin/measure/process/StreamingDqProcess.scala b/measure/src/main/scala/org/apache/griffin/measure/process/StreamingDqProcess.scala new file mode 100644 index 000000000..08c4b7aab --- /dev/null +++ b/measure/src/main/scala/org/apache/griffin/measure/process/StreamingDqProcess.scala @@ -0,0 +1,42 @@ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ +package org.apache.griffin.measure.process + +import org.apache.griffin.measure.config.params._ +import org.apache.griffin.measure.config.params.env._ +import org.apache.griffin.measure.config.params.user._ + +import scala.util.Try + +case class StreamingDqProcess(allParam: AllParam) extends DqProcess { + + val envParam: EnvParam = allParam.envParam + val userParam: UserParam = allParam.userParam + + def retriable: Boolean = true + + def init: Try[_] = Try {} + + def run: Try[_] = Try { + ; + } + + def end: Try[_] = Try {} + +} diff --git a/measure/src/main/scala/org/apache/griffin/measure/process/engine/DataFrameOprEngine.scala b/measure/src/main/scala/org/apache/griffin/measure/process/engine/DataFrameOprEngine.scala new file mode 100644 index 000000000..70d3213eb --- /dev/null +++ b/measure/src/main/scala/org/apache/griffin/measure/process/engine/DataFrameOprEngine.scala @@ -0,0 +1,74 @@ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ +package org.apache.griffin.measure.process.engine + +import org.apache.griffin.measure.config.params.user.DataSourceParam +import org.apache.griffin.measure.data.source.{DataSource, DataSourceFactory} +import org.apache.griffin.measure.persist.Persist +import org.apache.griffin.measure.rules.step._ +import org.apache.spark.sql.SQLContext +import org.apache.spark.streaming.StreamingContext + +case class DataFrameOprEngine(sqlContext: SQLContext, @transient ssc: StreamingContext) extends DqEngine { + + def genDataSource(dataSourceParam: DataSourceParam): Option[DataSource] = { + DataSourceFactory.genDataSource(sqlContext, ssc, dataSourceParam) + } + + def runRuleStep(ruleStep: ConcreteRuleStep): Boolean = { + ruleStep match { + case DfOprStep(name, rule, _) => { + try { +// val rdf = sqlContext.sql(rule) +// rdf.registerTempTable(name) + // fixme + true + } catch { + case e: Throwable => { + error(s"run rule ${name} error: ${e.getMessage}") + false + } + } + } + case _ => false + } + } + + def persistResult(ruleStep: ConcreteRuleStep, persist: Persist): Boolean = { + ruleStep match { + case SparkSqlStep(_, _, _) => { + try { + // fixme + true + } catch { + case e: Throwable => { + error(s"persist result ${ruleStep.name} error: ${e.getMessage}") + false + } + } + } + case _ => false + } + } + +} + + + + diff --git a/measure/src/main/scala/org/apache/griffin/measure/process/engine/DqEngine.scala b/measure/src/main/scala/org/apache/griffin/measure/process/engine/DqEngine.scala new file mode 100644 index 000000000..493539c4b --- /dev/null +++ b/measure/src/main/scala/org/apache/griffin/measure/process/engine/DqEngine.scala @@ -0,0 +1,35 @@ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ +package org.apache.griffin.measure.process.engine + +import org.apache.griffin.measure.config.params.user.DataSourceParam +import org.apache.griffin.measure.data.source.DataSource +import org.apache.griffin.measure.log.Loggable +import org.apache.griffin.measure.persist.Persist +import org.apache.griffin.measure.rules.step._ + +trait DqEngine extends Loggable with Serializable { + + def genDataSource(dataSourceParam: DataSourceParam): Option[DataSource] + + def runRuleStep(ruleStep: ConcreteRuleStep): Boolean + + def persistResult(ruleStep: ConcreteRuleStep, persist: Persist): Boolean + +} diff --git a/measure/src/main/scala/org/apache/griffin/measure/process/engine/DqEngineFactory.scala b/measure/src/main/scala/org/apache/griffin/measure/process/engine/DqEngineFactory.scala new file mode 100644 index 000000000..033c53e3e --- /dev/null +++ b/measure/src/main/scala/org/apache/griffin/measure/process/engine/DqEngineFactory.scala @@ -0,0 +1,47 @@ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ +package org.apache.griffin.measure.process.engine + +import org.apache.spark.sql.SQLContext +import org.apache.spark.streaming.StreamingContext + + +object DqEngineFactory { + + private val engineTypes = List("spark-sql", "df-opr") + + private final val SparkSqlEngineType = "spark-sql" + private final val DataFrameOprEngineType = "df-opr" + + def genDqEngines(sqlContext: SQLContext, ssc: StreamingContext): DqEngines = { + val engines = engineTypes.flatMap { et => + genDqEngine(et, sqlContext, ssc) + } + DqEngines(engines) + } + + private def genDqEngine(engineType: String, sqlContext: SQLContext, ssc: StreamingContext): Option[DqEngine] = { + engineType match { + case SparkSqlEngineType => Some(SparkSqlEngine(sqlContext, ssc)) + case DataFrameOprEngineType => Some(DataFrameOprEngine(sqlContext, ssc)) + case _ => None + } + } + +} diff --git a/measure/src/main/scala/org/apache/griffin/measure/process/engine/DqEngines.scala b/measure/src/main/scala/org/apache/griffin/measure/process/engine/DqEngines.scala new file mode 100644 index 000000000..d7ab619c3 --- /dev/null +++ b/measure/src/main/scala/org/apache/griffin/measure/process/engine/DqEngines.scala @@ -0,0 +1,74 @@ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ +package org.apache.griffin.measure.process.engine + +import org.apache.griffin.measure.config.params.user.DataSourceParam +import org.apache.griffin.measure.data.source._ +import org.apache.griffin.measure.log.Loggable +import org.apache.griffin.measure.persist.Persist +import org.apache.griffin.measure.rules.step._ + +case class DqEngines(engines: Seq[DqEngine]) extends DqEngine { + + def initDataSources(dataSourceParams: Seq[DataSourceParam]): Unit = { + val dataSources = dataSourceParams.flatMap { param => + genDataSource(param) + } + dataSources.foreach { ds => + ds.init + } + } + + def runRuleSteps(ruleSteps: Seq[ConcreteRuleStep]): Unit = { + ruleSteps.foreach { ruleStep => + runRuleStep(ruleStep) + } + } + + def persistResults(ruleSteps: Seq[ConcreteRuleStep], persist: Persist): Unit = { + ruleSteps.foreach { ruleStep => + persistResult(ruleStep, persist) + } + } + + def genDataSource(dataSourceParam: DataSourceParam): Option[DataSource] = { + val ret = engines.foldLeft(None: Option[DataSource]) { (dsOpt, engine) => + if (dsOpt.isEmpty) engine.genDataSource(dataSourceParam) else dsOpt + } + if (ret.isEmpty) warn(s"init data source warn: no dq engine support ${dataSourceParam}") + ret + } + + def runRuleStep(ruleStep: ConcreteRuleStep): Boolean = { + val ret = engines.foldLeft(false) { (done, engine) => + done || engine.runRuleStep(ruleStep) + } + if (!ret) warn(s"run rule step warn: no dq engine support ${ruleStep}") + ret + } + + def persistResult(ruleStep: ConcreteRuleStep, persist: Persist): Boolean = { + val ret = engines.foldLeft(false) { (done, engine) => + done || engine.persistResult(ruleStep, persist) + } + if (!ret) error(s"persist result warn: no dq engine support ${ruleStep}") + ret + } + +} diff --git a/measure/src/main/scala/org/apache/griffin/measure/process/engine/SparkRowFormatter.scala b/measure/src/main/scala/org/apache/griffin/measure/process/engine/SparkRowFormatter.scala new file mode 100644 index 000000000..6ed05592d --- /dev/null +++ b/measure/src/main/scala/org/apache/griffin/measure/process/engine/SparkRowFormatter.scala @@ -0,0 +1,62 @@ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ +package org.apache.griffin.measure.process.engine + +import org.apache.spark.sql.Row +import org.apache.spark.sql.types.{ArrayType, DataType, StructField, StructType} + +import scala.collection.mutable.ArrayBuffer + +object SparkRowFormatter { + + def formatRow(row: Row): Map[String, Any] = { + formatRowWithSchema(row, row.schema) + } + + private def formatRowWithSchema(row: Row, schema: StructType): Map[String, Any] = { + formatStruct(schema.fields, row) + } + + private def formatStruct(schema: Seq[StructField], r: Row) = { + val paired = schema.zip(r.toSeq) + paired.foldLeft(Map[String, Any]())((s, p) => s ++ formatItem(p)) + } + + private def formatItem(p: Pair[StructField, Any]): Map[String, Any] = { + p match { + case (sf, a) => + sf.dataType match { + case ArrayType(et, _) => + Map(sf.name -> (if (a == null) a else formatArray(et, a.asInstanceOf[ArrayBuffer[Any]]))) + case StructType(s) => + Map(sf.name -> (if (a == null) a else formatStruct(s, a.asInstanceOf[Row]))) + case _ => Map(sf.name -> a) + } + } + } + + private def formatArray(et: DataType, arr: ArrayBuffer[Any]): Seq[Any] = { + et match { + case StructType(s) => arr.map(e => formatStruct(s, e.asInstanceOf[Row])) + case ArrayType(t, _) => + arr.map(e => formatArray(t, e.asInstanceOf[ArrayBuffer[Any]])) + case _ => arr + } + } +} diff --git a/measure/src/main/scala/org/apache/griffin/measure/process/engine/SparkSqlEngine.scala b/measure/src/main/scala/org/apache/griffin/measure/process/engine/SparkSqlEngine.scala new file mode 100644 index 000000000..bfefeefef --- /dev/null +++ b/measure/src/main/scala/org/apache/griffin/measure/process/engine/SparkSqlEngine.scala @@ -0,0 +1,82 @@ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ +package org.apache.griffin.measure.process.engine + +import org.apache.griffin.measure.config.params.user.DataSourceParam +import org.apache.griffin.measure.data.source._ +import org.apache.griffin.measure.persist.Persist +import org.apache.griffin.measure.rules.step._ +import org.apache.griffin.measure.utils.JsonUtil +import org.apache.spark.sql.{DataFrame, SQLContext} +import org.apache.spark.streaming.StreamingContext + +case class SparkSqlEngine(sqlContext: SQLContext, @transient ssc: StreamingContext) extends DqEngine { + + def genDataSource(dataSourceParam: DataSourceParam): Option[DataSource] = { + DataSourceFactory.genDataSource(sqlContext, ssc, dataSourceParam) + } + + def runRuleStep(ruleStep: ConcreteRuleStep): Boolean = { + ruleStep match { + case SparkSqlStep(name, rule, _) => { + try { + val rdf = sqlContext.sql(rule) + rdf.registerTempTable(name) + true + } catch { + case e: Throwable => { + error(s"run rule ${name} error: ${e.getMessage}") + false + } + } + } + case _ => false + } + } + + def persistResult(ruleStep: ConcreteRuleStep, persist: Persist): Boolean = { + ruleStep match { + case SparkSqlStep(_, _, _) => { + try { + val pdf = sqlContext.sql(getDfSql(ruleStep.name)) + val records = pdf.toJSON.collect() + val persistType = ruleStep.persistType + // fixme + records.foreach(println) + true + } catch { + case e: Throwable => { + error(s"persist result ${ruleStep.name} error: ${e.getMessage}") + false + } + } + } + case _ => false + } + } + + private def getDfSql(name: String): String = { + s"SELECT * FROM `${name}`" + } + +} + + + + diff --git a/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/DataFrameOprAdaptor.scala b/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/DataFrameOprAdaptor.scala new file mode 100644 index 000000000..e4b75ed1c --- /dev/null +++ b/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/DataFrameOprAdaptor.scala @@ -0,0 +1,36 @@ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ +package org.apache.griffin.measure.rules.adaptor + +import org.apache.griffin.measure.rules.step._ + +case class DataFrameOprAdaptor() extends RuleAdaptor { + + def genRuleStep(param: Map[String, Any]): Seq[RuleStep] = { + DfOprStep(getName(param), getRule(param), getPersistType(param)) :: Nil + } + + def getTempSourceNames(param: Map[String, Any]): Seq[String] = { + param.get(_name) match { + case Some(name) => name.toString :: Nil + case _ => Nil + } + } + +} diff --git a/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/GriffinDslAdaptor.scala b/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/GriffinDslAdaptor.scala new file mode 100644 index 000000000..8d617081c --- /dev/null +++ b/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/GriffinDslAdaptor.scala @@ -0,0 +1,180 @@ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ +package org.apache.griffin.measure.rules.adaptor + +import org.apache.griffin.measure.config.params.user.RuleParam +import org.apache.griffin.measure.rules.dsl._ +import org.apache.griffin.measure.rules.dsl.analyzer._ +import org.apache.griffin.measure.rules.dsl.expr.Expr +import org.apache.griffin.measure.rules.dsl.parser.GriffinDslParser +import org.apache.griffin.measure.rules.step._ + +case class GriffinDslAdaptor(dataSourceNames: Seq[String], functionNames: Seq[String]) extends RuleAdaptor { + + object StepInfo { + val _Name = "name" + val _PersistType = "persist.type" + def getNameOpt(param: Map[String, Any]): Option[String] = param.get(_Name).flatMap(a => Some(a.toString)) + def getPersistType(param: Map[String, Any]): PersistType = PersistType(param.getOrElse(_PersistType, "").toString) + } + object AccuracyInfo { + val _Source = "source" + val _Target = "target" + val _MissRecord = "miss.record" + val _MissCount = "miss.count" + val _TotalCount = "total.count" + def getNameOpt(param: Map[String, Any], key: String): Option[String] = param.get(key).flatMap(a => Some(a.toString)) + def resultName(param: Map[String, Any], key: String): String = { + val nameOpt = param.get(key) match { + case Some(prm: Map[String, Any]) => StepInfo.getNameOpt(prm) + case _ => None + } + nameOpt.getOrElse(key) + } + def resultPersistType(param: Map[String, Any], key: String): PersistType = { + param.get(key) match { + case Some(prm: Map[String, Any]) => StepInfo.getPersistType(prm) + case _ => NonePersistType + } + } + } + + val _dqType = "dq.type" + val _details = "details" + + val filteredFunctionNames = functionNames.filter { fn => + fn.matches("""^[a-zA-Z_]\w*$""") + } + val parser = GriffinDslParser(dataSourceNames, filteredFunctionNames) + + protected def getDqType(param: Map[String, Any]) = DqType(param.getOrElse(_dqType, "").toString) + protected def getDetails(param: Map[String, Any]) = { + param.get(_details) match { + case Some(p: Map[String, Any]) => p + case _ => Map[String, Any]() + } + } + + def genRuleStep(param: Map[String, Any]): Seq[RuleStep] = { + GriffinDslStep(getName(param), getRule(param), getDqType(param), getDetails(param)) :: Nil + } + + def getTempSourceNames(param: Map[String, Any]): Seq[String] = { + val dqType = getDqType(param) + param.get(_name) match { + case Some(name) => { + dqType match { + case AccuracyType => { + Seq[String]( + AccuracyInfo.resultName(param, AccuracyInfo._MissRecord), + AccuracyInfo.resultName(param, AccuracyInfo._MissCount), + AccuracyInfo.resultName(param, AccuracyInfo._TotalCount) + ) + } + case ProfilingType => { + Nil + } + case TimelinessType => { + Nil + } + case _ => Nil + } + } + case _ => Nil + } + } + + override def adaptConcreteRuleStep(ruleStep: RuleStep): Seq[ConcreteRuleStep] = { + ruleStep match { + case rs @ GriffinDslStep(_, rule, _, _) => { + val exprOpt = try { + val result = parser.parseAll(parser.rootExpression, rule) + if (result.successful) Some(result.get) + else { + warn(s"adapt concrete rule step warn: ${rule}") + None + } + } catch { + case e: Throwable => { + error(s"adapt concrete rule step error: ${e.getMessage}") + None + } + } + + exprOpt match { + case Some(expr) => { + transConcreteRuleSteps(rs, expr) + } + case _ => Nil + } + } + case _ => Nil + } + } + + private def transConcreteRuleSteps(ruleStep: GriffinDslStep, expr: Expr): Seq[ConcreteRuleStep] = { + ruleStep.dqType match { + case AccuracyType => { + val sourceName = AccuracyInfo.getNameOpt(ruleStep.details, AccuracyInfo._Source) match { + case Some(name) => name + case _ => dataSourceNames.head + } + val targetName = AccuracyInfo.getNameOpt(ruleStep.details, AccuracyInfo._Target) match { + case Some(name) => name + case _ => dataSourceNames.tail.head + } + val analyzer = AccuracyAnalyzer(expr, sourceName, targetName) + + // 1. miss record + val missRecordSql = { + val selClause = analyzer.sourceSelectionExprs.map { sel => + val alias = sel.alias match { + case Some(a) => s" AS ${a}" + case _ => "" + } + s"${sel.desc}${alias}" + }.mkString(", ") + + s"SELECT ${selClause} FROM ${sourceName} LEFT JOIN ${targetName}" + } + println(missRecordSql) + + // 2. miss count + val missCountSql = { + "" + } + + // 3. total count + val totalCountSql = { + "" + } + + Nil + } + case ProfilingType => { + Nil + } + case TimelinessType => { + Nil + } + case _ => Nil + } + } + +} diff --git a/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/RuleAdaptor.scala b/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/RuleAdaptor.scala new file mode 100644 index 000000000..26b6ced62 --- /dev/null +++ b/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/RuleAdaptor.scala @@ -0,0 +1,65 @@ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ +package org.apache.griffin.measure.rules.adaptor + +import java.util.concurrent.atomic.AtomicLong + +import scala.collection.mutable.{Set => MutableSet} +import org.apache.griffin.measure.config.params.user._ +import org.apache.griffin.measure.log.Loggable +import org.apache.griffin.measure.rules.step.{ConcreteRuleStep, RuleStep} +import org.apache.griffin.measure.rules.dsl.{DslType, PersistType} + +trait RuleAdaptor extends Loggable with Serializable { + + val _name = "name" + val _rule = "rule" + val _persistType = "persist.type" + + protected def getName(param: Map[String, Any]) = param.getOrElse(_name, RuleStepNameGenerator.genName).toString + protected def getRule(param: Map[String, Any]) = param.getOrElse(_rule, "").toString + protected def getPersistType(param: Map[String, Any]) = PersistType(param.getOrElse(_persistType, "").toString) + + def getTempSourceNames(param: Map[String, Any]): Seq[String] + + def genRuleStep(param: Map[String, Any]): Seq[RuleStep] + def genConcreteRuleStep(param: Map[String, Any]): Seq[ConcreteRuleStep] = { + genRuleStep(param).flatMap { rs => + rs match { + case s: ConcreteRuleStep => Some(s) + case _ => adaptConcreteRuleStep(rs) + } + } + } + protected def adaptConcreteRuleStep(ruleStep: RuleStep): Seq[ConcreteRuleStep] = Nil + +} + +object RuleStepNameGenerator { + private val counter: AtomicLong = new AtomicLong(0L) + private val head: String = "rs" + + def genName: String = { + s"${head}${increment}" + } + + private def increment: Long = { + counter.incrementAndGet() + } +} \ No newline at end of file diff --git a/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/RuleAdaptorGroup.scala b/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/RuleAdaptorGroup.scala new file mode 100644 index 000000000..8a6b40f5d --- /dev/null +++ b/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/RuleAdaptorGroup.scala @@ -0,0 +1,93 @@ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ +package org.apache.griffin.measure.rules.adaptor + +import org.apache.griffin.measure.config.params.user.{EvaluateRuleParam, RuleParam} +import org.apache.griffin.measure.rules.dsl._ +import org.apache.griffin.measure.rules.step._ +import org.apache.spark.sql.SQLContext + +import scala.collection.mutable.{Map => MutableMap} + +object RuleAdaptorGroup { + + val _dslType = "dsl.type" + + var dataSourceNames: Seq[String] = _ + var functionNames: Seq[String] = _ + + def init(sqlContext: SQLContext, dsNames: Seq[String]): Unit = { + val functions = sqlContext.sql("show functions") + functionNames = functions.map(_.getString(0)).collect + dataSourceNames = dsNames + } + + private def getDslType(param: Map[String, Any], defDslType: DslType) = { + val dt = DslType(param.getOrElse(_dslType, "").toString) + dt match { + case UnknownDslType => defDslType + case _ => dt + } + } + + private def genRuleAdaptor(dslType: DslType, dataSourceNames: Seq[String]): Option[RuleAdaptor] = { + dslType match { + case SparkSqlType => Some(SparkSqlAdaptor()) + case DfOprType => Some(DataFrameOprAdaptor()) + case GriffinDslType => Some(GriffinDslAdaptor(dataSourceNames, functionNames)) + case _ => None + } + } + +// def genRuleSteps(evaluateRuleParam: EvaluateRuleParam): Seq[RuleStep] = { +// val dslTypeStr = if (evaluateRuleParam.dslType == null) "" else evaluateRuleParam.dslType +// val defaultDslType = DslType(dslTypeStr) +// val rules = evaluateRuleParam.rules +// var dsNames = dataSourceNames +// val steps = rules.flatMap { param => +// val dslType = getDslType(param) +// genRuleAdaptor(dslType) match { +// case Some(ruleAdaptor) => ruleAdaptor.genRuleStep(param) +// case _ => Nil +// } +// } +// steps.foreach(println) +// steps +// } + + def genConcreteRuleSteps(evaluateRuleParam: EvaluateRuleParam): Seq[ConcreteRuleStep] = { + val dslTypeStr = if (evaluateRuleParam.dslType == null) "" else evaluateRuleParam.dslType + val defaultDslType = DslType(dslTypeStr) + val ruleParams = evaluateRuleParam.rules + val (steps, dsNames) = ruleParams.foldLeft((Seq[ConcreteRuleStep](), dataSourceNames)) { (res, param) => + val (preSteps, preNames) = res + val dslType = getDslType(param, defaultDslType) + val (curSteps, curNames) = genRuleAdaptor(dslType, preNames) match { + case Some(ruleAdaptor) => (ruleAdaptor.genConcreteRuleStep(param), preNames ++ ruleAdaptor.getTempSourceNames(param)) + case _ => (Nil, preNames) + } + (preSteps ++ curSteps, curNames) + } + steps.foreach(println) + steps + } + + + +} diff --git a/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/SparkSqlAdaptor.scala b/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/SparkSqlAdaptor.scala new file mode 100644 index 000000000..4ae1129ef --- /dev/null +++ b/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/SparkSqlAdaptor.scala @@ -0,0 +1,38 @@ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ +package org.apache.griffin.measure.rules.adaptor + +import org.apache.griffin.measure.config.params.user.RuleParam +import org.apache.griffin.measure.rules.dsl.PersistType +import org.apache.griffin.measure.rules.step._ + +case class SparkSqlAdaptor() extends RuleAdaptor { + + def genRuleStep(param: Map[String, Any]): Seq[RuleStep] = { + SparkSqlStep(getName(param), getRule(param), getPersistType(param)) :: Nil + } + + def getTempSourceNames(param: Map[String, Any]): Seq[String] = { + param.get(_name) match { + case Some(name) => name.toString :: Nil + case _ => Nil + } + } + +} diff --git a/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/DqType.scala b/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/DqType.scala new file mode 100644 index 000000000..b250cb9fd --- /dev/null +++ b/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/DqType.scala @@ -0,0 +1,58 @@ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ +package org.apache.griffin.measure.rules.dsl + +import scala.util.matching.Regex + + +sealed trait DqType { + val regex: Regex + val desc: String +} + +object DqType { + private val dqTypes: List[DqType] = List(AccuracyType, ProfilingType, TimelinessType, UnknownType) + def apply(ptn: String): DqType = { + dqTypes.filter(tp => ptn match { + case tp.regex() => true + case _ => false + }).headOption.getOrElse(UnknownType) + } + def unapply(pt: DqType): Option[String] = Some(pt.desc) +} + +final case object AccuracyType extends DqType { + val regex = "^(?i)accuracy$".r + val desc = "accuracy" +} + +final case object ProfilingType extends DqType { + val regex = "^(?i)profiling$".r + val desc = "profiling$" +} + +final case object TimelinessType extends DqType { + val regex = "^(?i)timeliness$".r + val desc = "timeliness" +} + +final case object UnknownType extends DqType { + val regex = "".r + val desc = "unknown" +} \ No newline at end of file diff --git a/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/DslType.scala b/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/DslType.scala new file mode 100644 index 000000000..8e9e8edb2 --- /dev/null +++ b/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/DslType.scala @@ -0,0 +1,58 @@ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ +package org.apache.griffin.measure.rules.dsl + +import scala.util.matching.Regex + + +sealed trait DslType { + val regex: Regex + val desc: String +} + +object DslType { + private val dslTypes: List[DslType] = List(SparkSqlType, GriffinDslType, DfOprType, UnknownDslType) + def apply(ptn: String): DslType = { + dslTypes.filter(tp => ptn match { + case tp.regex() => true + case _ => false + }).headOption.getOrElse(UnknownDslType) + } + def unapply(pt: DslType): Option[String] = Some(pt.desc) +} + +final case object SparkSqlType extends DslType { + val regex = "^(?i)spark-?sql$".r + val desc = "spark-sql" +} + +final case object DfOprType extends DslType { + val regex = "^(?i)df-?opr$".r + val desc = "df-opr" +} + +final case object GriffinDslType extends DslType { + val regex = "^(?i)griffin-?dsl$".r + val desc = "griffin-dsl" +} + +final case object UnknownDslType extends DslType { + val regex = "".r + val desc = "unknown" +} \ No newline at end of file diff --git a/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/PersistType.scala b/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/PersistType.scala new file mode 100644 index 000000000..b8f3c4653 --- /dev/null +++ b/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/PersistType.scala @@ -0,0 +1,58 @@ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ +package org.apache.griffin.measure.rules.dsl + +import scala.util.matching.Regex + +sealed trait PersistType { + val regex: Regex + val desc: String +// def temp: Boolean = false +// def persist: Boolean = false +// def collect: Boolean = false +} + +object PersistType { + private val persistTypes: List[PersistType] = List(RecordPersistType, MetricPersistType, NonePersistType) + def apply(ptn: String): PersistType = { + persistTypes.filter(tp => ptn match { + case tp.regex() => true + case _ => false + }).headOption.getOrElse(NonePersistType) + } + def unapply(pt: PersistType): Option[String] = Some(pt.desc) +} + +final case object NonePersistType extends PersistType { + val regex: Regex = "".r + val desc: String = "none" +} + +final case object RecordPersistType extends PersistType { + val regex: Regex = "^(?i)record$".r + val desc: String = "record" +// override def temp: Boolean = true +} + +final case object MetricPersistType extends PersistType { + val regex: Regex = "^(?i)metric$".r + val desc: String = "metric" +// override def temp: Boolean = true +// override def collect: Boolean = true +} \ No newline at end of file diff --git a/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/analyzer/AccuracyAnalyzer.scala b/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/analyzer/AccuracyAnalyzer.scala new file mode 100644 index 000000000..6ef690f28 --- /dev/null +++ b/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/analyzer/AccuracyAnalyzer.scala @@ -0,0 +1,37 @@ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ +package org.apache.griffin.measure.rules.dsl.analyzer + +import org.apache.griffin.measure.rules.dsl.expr.Expr + + +case class AccuracyAnalyzer(expr: Expr, sourceName: String, targetName: String) extends BasicAnalyzer { + + val dataSourceNames = expr.preOrderTraverseDepthFirst(Set[String]())(seqDataSourceNames, combDataSourceNames) + + val sourceSelectionExprs = { + val seq = seqSelectionExprs(sourceName) + expr.preOrderTraverseDepthFirst(Seq[Expr]())(seq, combSelectionExprs) + } + val targetSelectionExprs = { + val seq = seqSelectionExprs(targetName) + expr.preOrderTraverseDepthFirst(Seq[Expr]())(seq, combSelectionExprs) + } + +} diff --git a/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/analyzer/BasicAnalyzer.scala b/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/analyzer/BasicAnalyzer.scala new file mode 100644 index 000000000..2ab2a5bdb --- /dev/null +++ b/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/analyzer/BasicAnalyzer.scala @@ -0,0 +1,49 @@ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ +package org.apache.griffin.measure.rules.dsl.analyzer + +import org.apache.griffin.measure.rules.dsl.expr._ + + +trait BasicAnalyzer extends Serializable { + + val expr: Expr + + val seqDataSourceNames = (expr: Expr, v: Set[String]) => { + expr match { + case DataSourceHeadExpr(name) => v + name + case _ => v + } + } + val combDataSourceNames = (a: Set[String], b: Set[String]) => a ++ b + + val seqSelectionExprs = (dsName: String) => (expr: Expr, v: Seq[Expr]) => { + expr match { + case se @ SelectionExpr(head: DataSourceHeadExpr, _) => { + head.alias match { + case Some(a) if (a == dsName) => v :+ se + case _ => v + } + } + case _ => v + } + } + val combSelectionExprs = (a: Seq[Expr], b: Seq[Expr]) => a ++ b + +} diff --git a/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/expr/AliasExpr.scala b/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/expr/AliasExpr.scala new file mode 100644 index 000000000..eba1db3b2 --- /dev/null +++ b/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/expr/AliasExpr.scala @@ -0,0 +1,27 @@ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ +package org.apache.griffin.measure.rules.dsl.expr + +case class AliasExpr(expr: Expr, aliasName: String) extends Expr { + + addChild(expr) + + def desc: String = expr.desc + def alias: Option[String] = Some(aliasName) +} \ No newline at end of file diff --git a/measure/src/main/scala/org/apache/griffin/measure/connector/streaming/StreamingDataConnector.scala b/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/expr/Expr.scala similarity index 71% rename from measure/src/main/scala/org/apache/griffin/measure/connector/streaming/StreamingDataConnector.scala rename to measure/src/main/scala/org/apache/griffin/measure/rules/dsl/expr/Expr.scala index c37caac15..08406fa48 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/connector/streaming/StreamingDataConnector.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/expr/Expr.scala @@ -16,19 +16,12 @@ KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License. */ -package org.apache.griffin.measure.connector.streaming +package org.apache.griffin.measure.rules.dsl.expr -import org.apache.griffin.measure.connector.DataConnector -import org.apache.spark.streaming.dstream.InputDStream +trait Expr extends TreeNode with Serializable { -import scala.util.Try + def desc: String - -trait StreamingDataConnector extends DataConnector { - - type K - type V - - def stream(): Try[InputDStream[(K, V)]] + def alias: Option[String] } diff --git a/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/expr/FunctionExpr.scala b/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/expr/FunctionExpr.scala new file mode 100644 index 000000000..19af690b2 --- /dev/null +++ b/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/expr/FunctionExpr.scala @@ -0,0 +1,29 @@ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ +package org.apache.griffin.measure.rules.dsl.expr + +case class FunctionExpr(functionName: String, args: Seq[Expr]) extends Expr { + + addChildren(args) + + def desc: String = { + s"${functionName}(${args.map(_.desc).mkString(", ")})" + } + def alias: Option[String] = None +} \ No newline at end of file diff --git a/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/expr/LiteralExpr.scala b/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/expr/LiteralExpr.scala new file mode 100644 index 000000000..96532edae --- /dev/null +++ b/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/expr/LiteralExpr.scala @@ -0,0 +1,72 @@ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ +package org.apache.griffin.measure.rules.dsl.expr + +import org.apache.griffin.measure.utils.TimeUtil + +trait LiteralExpr extends Expr { + def alias: Option[String] = None +} + +case class LiteralNullExpr(str: String) extends LiteralExpr { + def desc: String = "NULL" +} + +case class LiteralNanExpr(str: String) extends LiteralExpr { + def desc: String = "NaN" +} + +case class LiteralStringExpr(str: String) extends LiteralExpr { + def desc: String = str +} + +case class LiteralNumberExpr(str: String) extends LiteralExpr { + def desc: String = { + try { + if (str.contains(".")) { + str.toDouble.toString + } else { + str.toLong.toString + } + } catch { + case e: Throwable => throw new Exception(s"${str} is invalid number") + } + } +} + +case class LiteralTimeExpr(str: String) extends LiteralExpr { + def desc: String = { + TimeUtil.milliseconds(str) match { + case Some(t) => t.toString + case _ => throw new Exception(s"${str} is invalid time") + } + } +} + +case class LiteralBooleanExpr(str: String) extends LiteralExpr { + final val TrueRegex = """(?i)true""".r + final val FalseRegex = """(?i)false""".r + def desc: String = { + str match { + case TrueRegex() => true.toString + case FalseRegex() => false.toString + case _ => throw new Exception(s"${str} is invalid boolean") + } + } +} \ No newline at end of file diff --git a/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/expr/LogicalExpr.scala b/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/expr/LogicalExpr.scala new file mode 100644 index 000000000..155d2f274 --- /dev/null +++ b/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/expr/LogicalExpr.scala @@ -0,0 +1,129 @@ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ +package org.apache.griffin.measure.rules.dsl.expr + +trait LogicalExpr extends Expr { + def alias: Option[String] = None +} + +case class InExpr(head: Expr, is: Boolean, range: Seq[Expr]) extends LogicalExpr { + + addChildren(head +: range) + + def desc: String = { + val notStr = if (is) "" else " NOT" + s"${head.desc}${notStr} IN (${range.map(_.desc).mkString(", ")})" + } +} + +case class BetweenExpr(head: Expr, is: Boolean, range: Seq[Expr]) extends LogicalExpr { + + range match { + case first :: second :: _ => addChildren(head :: first :: second :: Nil) + case _ => throw new Exception("between expression exception: range less than 2") + } + + def desc: String = { + val notStr = if (is) "" else " NOT" + val rangeStr = range match { + case first :: second :: _ => s"${first.desc} AND ${second.desc}" + case _ => throw new Exception("between expression exception: range less than 2") + } + s"${head.desc}${notStr} BETWEEN ${rangeStr}" + } +} + +case class LikeExpr(head: Expr, is: Boolean, value: Expr) extends LogicalExpr { + + addChildren(head :: value :: Nil) + + def desc: String = { + val notStr = if (is) "" else " NOT" + s"${head.desc}${notStr} LIKE ${value.desc}" + } +} + +case class IsNullExpr(head: Expr, is: Boolean) extends LogicalExpr { + + addChild(head) + + def desc: String = { + val notStr = if (is) "" else " NOT" + s"${head.desc} IS${notStr} NULL" + } +} + +case class IsNanExpr(head: Expr, is: Boolean) extends LogicalExpr { + + addChild(head) + + def desc: String = { + val notStr = if (is) "" else "NOT " + s"${notStr}isnan(${head.desc})" + } +} + +// ----------- + +case class LogicalFactorExpr(factor: Expr, withBracket: Boolean) extends LogicalExpr { + + addChild(factor) + + def desc: String = { + if (withBracket) s"(${factor.desc})" else factor.desc + } + override def alias: Option[String] = factor.alias +} + +case class UnaryLogicalExpr(oprs: Seq[String], factor: LogicalExpr) extends LogicalExpr { + + addChild(factor) + + def desc: String = { + oprs.foldRight(factor.desc) { (opr, fac) => + s"(${trans(opr)} ${fac})" + } + } + private def trans(s: String): String = { + s match { + case "!" => "NOT" + case _ => s.toUpperCase + } + } +} + +case class BinaryLogicalExpr(factor: LogicalExpr, tails: Seq[(String, LogicalExpr)]) extends LogicalExpr { + + addChildren(factor +: tails.map(_._2)) + + def desc: String = { + val res = tails.foldLeft(factor.desc) { (fac, tail) => + val (opr, expr) = tail + s"${fac} ${trans(opr)} ${expr.desc}" + } + if (tails.size <= 0) res else s"${res}" + } + private def trans(s: String): String = { + s match { + case "&&" => "AND" + case "||" => "OR" + case _ => s.toUpperCase + } + } +} \ No newline at end of file diff --git a/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/expr/MathExpr.scala b/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/expr/MathExpr.scala new file mode 100644 index 000000000..12c51d8ff --- /dev/null +++ b/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/expr/MathExpr.scala @@ -0,0 +1,57 @@ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ +package org.apache.griffin.measure.rules.dsl.expr + +trait MathExpr extends Expr { + def alias: Option[String] = None +} + +case class MathFactorExpr(factor: Expr, withBracket: Boolean) extends MathExpr { + + addChild(factor) + + def desc: String = { + if (withBracket) s"(${factor.desc})" else factor.desc + } + override def alias: Option[String] = factor.alias +} + +case class UnaryMathExpr(oprs: Seq[String], factor: MathExpr) extends MathExpr { + + addChild(factor) + + def desc: String = { + oprs.foldRight(factor.desc) { (opr, fac) => + s"(${opr}${fac})" + } + } +} + +case class BinaryMathExpr(factor: MathExpr, tails: Seq[(String, MathExpr)]) extends MathExpr { + + addChildren(factor +: tails.map(_._2)) + + def desc: String = { + val res = tails.foldLeft(factor.desc) { (fac, tail) => + val (opr, expr) = tail + s"${fac} ${opr} ${expr.desc}" + } + if (tails.size <= 0) res else s"${res}" + } +} \ No newline at end of file diff --git a/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/expr/SelectExpr.scala b/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/expr/SelectExpr.scala new file mode 100644 index 000000000..d8066dbe7 --- /dev/null +++ b/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/expr/SelectExpr.scala @@ -0,0 +1,88 @@ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ +package org.apache.griffin.measure.rules.dsl.expr + +trait HeadExpr extends Expr { + +} + +case class DataSourceHeadExpr(name: String) extends HeadExpr { + def desc: String = name + def alias: Option[String] = Some(name) +} + +case class OtherHeadExpr(expr: Expr) extends HeadExpr { + + addChild(expr) + + def desc: String = expr.desc + def alias: Option[String] = expr.alias +} + +// ------------- + +trait SelectExpr extends Expr { + +} + +case class FieldSelectExpr(field: String) extends SelectExpr { + def desc: String = s".${field}" + def alias: Option[String] = Some(field) +} + +case class IndexSelectExpr(index: Expr) extends SelectExpr { + + addChild(index) + + def desc: String = s"[${index.desc}]" + def alias: Option[String] = None +} + +case class FunctionSelectExpr(functionName: String, args: Seq[Expr]) extends SelectExpr { + + addChildren(args) + + def desc: String = "" + def alias: Option[String] = None +} + +// ------------- + +case class SelectionExpr(head: HeadExpr, selectors: Seq[SelectExpr]) extends Expr { + + addChildren(head +: selectors) + + def desc: String = { + selectors.foldLeft(head.desc) { (hd, sel) => + sel match { + case FunctionSelectExpr(funcName, args) => { + val nargs = hd +: args.map(_.desc) + s"${funcName}(${nargs.mkString(", ")})" + } + case _ => s"${hd}${sel.desc}" + } + } + } + def alias: Option[String] = { + selectors.lastOption match { + case Some(last) => last.alias + case _ => head.alias + } + } +} \ No newline at end of file diff --git a/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/expr/TreeNode.scala b/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/expr/TreeNode.scala new file mode 100644 index 000000000..d9bb85a13 --- /dev/null +++ b/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/expr/TreeNode.scala @@ -0,0 +1,45 @@ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ +package org.apache.griffin.measure.rules.dsl.expr + +trait TreeNode extends Serializable { + + var children = Seq[TreeNode]() + + def addChild(expr: TreeNode) = { children :+= expr } + def addChildren(exprs: Seq[TreeNode]) = { children ++= exprs } + + def preOrderTraverseDepthFirst[T, A <: TreeNode](z: T)(seqOp: (A, T) => T, combOp: (T, T) => T): T = { + if (this.isInstanceOf[A]) { + val tv = seqOp(this.asInstanceOf[A], z) + children.foldLeft(combOp(z, tv)) { (ov, tn) => + combOp(ov, tn.preOrderTraverseDepthFirst(z)(seqOp, combOp)) + } + } else z + } + def postOrderTraverseDepthFirst[T, A <: TreeNode](z: T)(seqOp: (A, T) => T, combOp: (T, T) => T): T = { + if (this.isInstanceOf[A]) { + val cv = children.foldLeft(z) { (ov, tn) => + combOp(ov, tn.postOrderTraverseDepthFirst(z)(seqOp, combOp)) + } + combOp(z, seqOp(this.asInstanceOf[A], cv)) + } else z + } + +} diff --git a/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/parser/BasicParser.scala b/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/parser/BasicParser.scala new file mode 100644 index 000000000..bc2de36c4 --- /dev/null +++ b/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/parser/BasicParser.scala @@ -0,0 +1,291 @@ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ +package org.apache.griffin.measure.rules.dsl.parser + +import org.apache.griffin.measure.rules.dsl.expr._ + +import scala.util.parsing.combinator.JavaTokenParsers + +trait BasicParser extends JavaTokenParsers with Serializable { + + def rootExpression = expression + + /** + * BNF for basic parser + * + * -- literal -- + * ::= | | | | | + * ::= + * ::= | + * ::= ("d"|"h"|"m"|"s"|"ms") + * ::= true | false + * ::= null + * ::= nan + * + * -- selection -- + * ::= [ | | ]* + * ::= ("data source name registered") | + * ::= "." | "[" "]" + * ::= "[" "]" + * ::= "." "(" []? [, ]* ")" + * ::= + * + * -- math expr -- + * ::= | | | | "(" ")" + * ::= []* + * ::= [ ]+ + * ::= + * + * -- logical expr -- + * ::= []? + * ::= []? ( | ) + * ::= "(" []? [, ]+ ")" + * ::= []? + * ::= []? + * ::= []? + * + * ::= | | | | | | "(" ")" + * ::= []* + * ::= [ ]+ + * ::= + * + * -- expression -- + * = | + * + * -- function expr -- + * ::= "(" [] [, ]+ ")" + * ::= ("function name registered") + * ::= + * + * -- alias expr -- + * = + */ + + val dataSourceNames: Seq[String] + val functionNames: Seq[String] + + protected def genNamesParser(names: Seq[String]): Parser[String] = { + names.map { + fn => s"""${fn}""".r: Parser[String] + }.reduce(_ | _) + } + + object Literal { + val NULL: Parser[String] = """(?i)null""".r + val NAN: Parser[String] = """(?i)nan""".r + } + import Literal._ + + object Operator { + val MATH_UNARY: Parser[String] = "+" | "-" + val MATH_BINARIES: Seq[Parser[String]] = Seq(("*" | "/" | "%"), ("+" | "-")) + + val NOT: Parser[String] = """(?i)not""".r | "!" + val AND: Parser[String] = """(?i)and""".r | "&&" + val OR: Parser[String] = """(?i)or""".r | "||" + val IN: Parser[String] = """(?i)in""".r + val BETWEEN: Parser[String] = """(?i)between""".r + val AND_ONLY: Parser[String] = """(?i)and""".r + val IS: Parser[String] = """(?i)is""".r + val LIKE: Parser[String] = """(?i)like""".r + val COMPARE: Parser[String] = "=" | "!=" | "<>" | "<=" | ">=" | "<" | ">" + val LOGICAL_UNARY: Parser[String] = NOT + val LOGICAL_BINARIES: Seq[Parser[String]] = Seq((COMPARE), (AND), (OR)) + + val LSQBR: Parser[String] = "[" + val RSQBR: Parser[String] = "]" + val LBR: Parser[String] = "(" + val RBR: Parser[String] = ")" + + val DOT: Parser[String] = "." + val ALLSL: Parser[String] = "*" + val SQUOTE: Parser[String] = "'" + val DQUOTE: Parser[String] = "\"" + val UQUOTE: Parser[String] = "`" + val COMMA: Parser[String] = "," + + val AS: Parser[String] = "(?i)as".r + } + import Operator._ + + object Strings { + def AnyString: Parser[String] = """"(?:[^\"]|[\\][\"])*"""".r | """'(?:[^']|[\\]['])*'""".r + def UQuoteTableFieldName: Parser[String] = """`(?:[^`]|[\\][`])*`""".r + def TableFieldName: Parser[String] = UQuoteTableFieldName | """[a-zA-Z_]\w*""".r + def DataSourceName: Parser[String] = genNamesParser(dataSourceNames) + def FunctionName: Parser[String] = genNamesParser(functionNames) + + def IntegerNumber: Parser[String] = """[+\-]?\d+""".r + def DoubleNumber: Parser[String] = """[+\-]?(\.\d+|\d+\.\d*)""".r + def IndexNumber: Parser[String] = IntegerNumber + + def TimeString: Parser[String] = """([+\-]?\d+)(d|h|m|s|ms)""".r + def BooleanString: Parser[String] = """(?i)true|false""".r + } + import Strings._ + + /** + * -- literal -- + * ::= | | | | | + * ::= + * ::= | + * ::= ("d"|"h"|"m"|"s"|"ms") + * ::= true | false + * ::= null + * ::= nan + */ + def literal: Parser[LiteralExpr] = literalNull | literalNan | literalBoolean | literalString | literalTime | literalNumber + def literalNull: Parser[LiteralNullExpr] = NULL ^^ { LiteralNullExpr(_) } + def literalNan: Parser[LiteralNanExpr] = NAN ^^ { LiteralNanExpr(_) } + def literalString: Parser[LiteralStringExpr] = AnyString ^^ { LiteralStringExpr(_) } + def literalNumber: Parser[LiteralNumberExpr] = (DoubleNumber | IntegerNumber) ^^ { LiteralNumberExpr(_) } + def literalTime: Parser[LiteralTimeExpr] = TimeString ^^ { LiteralTimeExpr(_) } + def literalBoolean: Parser[LiteralBooleanExpr] = BooleanString ^^ { LiteralBooleanExpr(_) } + + /** + * -- selection -- + * ::= [ | | ]* + * ::= ("data source name registered") | + * ::= "." | "[" "]" + * ::= "[" "]" + * ::= "." "(" []? [, ]* ")" + * ::= + */ + + def selection: Parser[SelectionExpr] = selectionHead ~ rep(selector) ^^ { + case head ~ sels => SelectionExpr(head, sels) + } + def selectionHead: Parser[HeadExpr] = DataSourceName ^^ { DataSourceHeadExpr(_) } | function ^^ { OtherHeadExpr(_) } + def selector: Parser[SelectExpr] = functionSelect | fieldSelect | indexSelect + def fieldSelect: Parser[FieldSelectExpr] = DOT ~> TableFieldName ^^ { FieldSelectExpr(_) } + def indexSelect: Parser[IndexSelectExpr] = LSQBR ~> argument <~ RSQBR ^^ { IndexSelectExpr(_) } + def functionSelect: Parser[FunctionSelectExpr] = DOT ~ FunctionName ~ LBR ~ repsep(argument, COMMA) ~ RBR ^^ { + case _ ~ name ~ _ ~ args ~ _ => FunctionSelectExpr(name, args) + } + + /** + * -- math expr -- + * ::= | | | | "(" ")" + * ::= []* + * ::= [ ]+ + * ::= + */ + + def mathFactor: Parser[MathExpr] = (literal | aliasExpr | function | selection) ^^ { + MathFactorExpr(_, false) + } | LBR ~> mathExpression <~ RBR ^^ { + MathFactorExpr(_, true) + } + def unaryMathExpression: Parser[MathExpr] = rep(MATH_UNARY) ~ mathFactor ^^ { + case Nil ~ a => a + case list ~ a => UnaryMathExpr(list, a) + } + def binaryMathExpressions: Seq[Parser[MathExpr]] = + MATH_BINARIES.foldLeft(List[Parser[MathExpr]](unaryMathExpression)) { (parsers, binaryParser) => + val pre = parsers.head + val cur = pre ~ rep(binaryParser ~ pre) ^^ { + case a ~ Nil => a + case a ~ list => BinaryMathExpr(a, list.map(c => (c._1, c._2))) + } + cur :: parsers + } + def mathExpression: Parser[MathExpr] = binaryMathExpressions.head + + /** + * -- logical expr -- + * ::= []? + * ::= []? ( | ) + * ::= "(" []? [, ]+ ")" + * ::= []? + * ::= []? + * ::= []? + * + * ::= | | | | | | "(" ")" + * ::= []* + * ::= [ ]+ + * ::= + */ + + def inExpr: Parser[LogicalExpr] = mathExpression ~ opt(NOT) ~ IN ~ LBR ~ repsep(mathExpression, COMMA) ~ RBR ^^ { + case head ~ notOpt ~ _ ~ _ ~ list ~ _ => InExpr(head, notOpt.isEmpty, list) + } + def betweenExpr: Parser[LogicalExpr] = mathExpression ~ opt(NOT) ~ BETWEEN ~ LBR ~ repsep(mathExpression, COMMA) ~ RBR ^^ { + case head ~ notOpt ~ _ ~ _ ~ list ~ _ => BetweenExpr(head, notOpt.isEmpty, list) + } | mathExpression ~ opt(NOT) ~ BETWEEN ~ mathExpression ~ AND_ONLY ~ mathExpression ^^ { + case head ~ notOpt ~ _ ~ first ~ _ ~ second => BetweenExpr(head, notOpt.isEmpty, Seq(first, second)) + } + def likeExpr: Parser[LogicalExpr] = mathExpression ~ opt(NOT) ~ LIKE ~ mathExpression ^^ { + case head ~ notOpt ~ _ ~ value => LikeExpr(head, notOpt.isEmpty, value) + } + def isNullExpr: Parser[LogicalExpr] = mathExpression ~ IS ~ opt(NOT) ~ NULL ^^ { + case head ~ _ ~ notOpt ~ _ => IsNullExpr(head, notOpt.isEmpty) + } + def isNanExpr: Parser[LogicalExpr] = mathExpression ~ IS ~ opt(NOT) ~ NAN ^^ { + case head ~ _ ~ notOpt ~ _ => IsNanExpr(head, notOpt.isEmpty) + } + + def logicalFactor: Parser[LogicalExpr] = (inExpr | betweenExpr | likeExpr | isNullExpr | isNanExpr | mathExpression) ^^ { + LogicalFactorExpr(_, false) + } | LBR ~> logicalExpression <~ RBR ^^ { + LogicalFactorExpr(_, true) + } + def unaryLogicalExpression: Parser[LogicalExpr] = rep(LOGICAL_UNARY) ~ logicalFactor ^^ { + case Nil ~ a => a + case list ~ a => UnaryLogicalExpr(list, a) + } + def binaryLogicalExpressions: Seq[Parser[LogicalExpr]] = + LOGICAL_BINARIES.foldLeft(List[Parser[LogicalExpr]](unaryLogicalExpression)) { (parsers, binaryParser) => + val pre = parsers.head + val cur = pre ~ rep(binaryParser ~ pre) ^^ { + case a ~ Nil => a + case a ~ list => BinaryLogicalExpr(a, list.map(c => (c._1, c._2))) + } + cur :: parsers + } + def logicalExpression: Parser[LogicalExpr] = binaryLogicalExpressions.head + + /** + * -- expression -- + * = | + */ + + def expression: Parser[Expr] = logicalExpression | mathExpression + + /** + * -- function expr -- + * ::= "(" [] [, ]+ ")" + * ::= ("function name registered") + * ::= + */ + + def function: Parser[FunctionExpr] = FunctionName ~ LBR ~ repsep(argument, COMMA) ~ RBR ^^ { + case name ~ _ ~ args ~ _ => FunctionExpr(name, args) + } + def argument: Parser[Expr] = expression + + /** + * -- alias expr -- + * = + */ + + def aliasExpr: Parser[AliasExpr] = LBR ~> expression ~ AS ~ TableFieldName <~ RBR ^^ { + case expr ~ _ ~ aname => AliasExpr(expr, aname) + } + +} diff --git a/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/parser/GriffinDslParser.scala b/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/parser/GriffinDslParser.scala new file mode 100644 index 000000000..5ab1a11e2 --- /dev/null +++ b/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/parser/GriffinDslParser.scala @@ -0,0 +1,23 @@ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ +package org.apache.griffin.measure.rules.dsl.parser + +case class GriffinDslParser(dataSourceNames: Seq[String], functionNames: Seq[String]) extends BasicParser { + +} diff --git a/measure/src/main/scala/org/apache/griffin/measure/connector/cache/CacheDataConnector.scala b/measure/src/main/scala/org/apache/griffin/measure/rules/step/ConcreteRuleStep.scala similarity index 68% rename from measure/src/main/scala/org/apache/griffin/measure/connector/cache/CacheDataConnector.scala rename to measure/src/main/scala/org/apache/griffin/measure/rules/step/ConcreteRuleStep.scala index 1dfe8e2da..ddb6099cd 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/connector/cache/CacheDataConnector.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rules/step/ConcreteRuleStep.scala @@ -16,18 +16,12 @@ KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License. */ -package org.apache.griffin.measure.connector.cache +package org.apache.griffin.measure.rules.step -import org.apache.griffin.measure.connector.DataConnector -import org.apache.spark.rdd.RDD +import org.apache.griffin.measure.rules.dsl.{DslType, PersistType} -import scala.util.Try +trait ConcreteRuleStep extends RuleStep { -trait CacheDataConnector extends DataConnector with DataCacheable with DataUpdatable { - - def saveData(rdd: RDD[Map[String, Any]], ms: Long): Unit - - def readData(): Try[RDD[Map[String, Any]]] + val persistType: PersistType } - diff --git a/measure/src/main/scala/org/apache/griffin/measure/rules/step/DfOprStep.scala b/measure/src/main/scala/org/apache/griffin/measure/rules/step/DfOprStep.scala new file mode 100644 index 000000000..0581c1781 --- /dev/null +++ b/measure/src/main/scala/org/apache/griffin/measure/rules/step/DfOprStep.scala @@ -0,0 +1,27 @@ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ +package org.apache.griffin.measure.rules.step + +import org.apache.griffin.measure.rules.dsl._ + +case class DfOprStep(name: String, rule: String, persistType: PersistType) extends ConcreteRuleStep { + + val dslType: DslType = DfOprType + +} diff --git a/measure/src/main/scala/org/apache/griffin/measure/rules/step/GriffinDslStep.scala b/measure/src/main/scala/org/apache/griffin/measure/rules/step/GriffinDslStep.scala new file mode 100644 index 000000000..057b18b33 --- /dev/null +++ b/measure/src/main/scala/org/apache/griffin/measure/rules/step/GriffinDslStep.scala @@ -0,0 +1,27 @@ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ +package org.apache.griffin.measure.rules.step + +import org.apache.griffin.measure.rules.dsl._ + +case class GriffinDslStep(name: String, rule: String, dqType: DqType, details: Map[String, Any]) extends RuleStep { + + val dslType: DslType = GriffinDslType + +} diff --git a/measure/src/main/scala/org/apache/griffin/measure/connector/cache/DataUpdatable.scala b/measure/src/main/scala/org/apache/griffin/measure/rules/step/RuleStep.scala similarity index 73% rename from measure/src/main/scala/org/apache/griffin/measure/connector/cache/DataUpdatable.scala rename to measure/src/main/scala/org/apache/griffin/measure/rules/step/RuleStep.scala index 07c818705..e53f07748 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/connector/cache/DataUpdatable.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rules/step/RuleStep.scala @@ -16,15 +16,15 @@ KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License. */ -package org.apache.griffin.measure.connector.cache +package org.apache.griffin.measure.rules.step -import org.apache.spark.rdd.RDD +import org.apache.griffin.measure.rules.dsl.{DslType, PersistType} -trait DataUpdatable { +trait RuleStep extends Serializable { - def cleanOldData(): Unit = {} + val dslType: DslType - def updateOldData(t: Long, oldData: Iterable[Map[String, Any]]): Unit = {} - def updateAllOldData(oldRdd: RDD[Map[String, Any]]): Unit = {} + val name: String + val rule: String } diff --git a/measure/src/main/scala/org/apache/griffin/measure/rules/step/SparkSqlStep.scala b/measure/src/main/scala/org/apache/griffin/measure/rules/step/SparkSqlStep.scala new file mode 100644 index 000000000..8fcb5f774 --- /dev/null +++ b/measure/src/main/scala/org/apache/griffin/measure/rules/step/SparkSqlStep.scala @@ -0,0 +1,28 @@ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ +package org.apache.griffin.measure.rules.step + +import org.apache.griffin.measure.persist._ +import org.apache.griffin.measure.rules.dsl._ + +case class SparkSqlStep(name: String, rule: String, persistType: PersistType) extends ConcreteRuleStep { + + val dslType: DslType = SparkSqlType + +} diff --git a/measure/src/main/scala/org/apache/griffin/measure/utils/TimeUtil.scala b/measure/src/main/scala/org/apache/griffin/measure/utils/TimeUtil.scala index 0079d104c..fe721d2bd 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/utils/TimeUtil.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/utils/TimeUtil.scala @@ -22,8 +22,8 @@ import scala.util.{Failure, Success, Try} object TimeUtil { - final val TimeRegex = """([+\-]?\d+)(d|h|m|s|ms)""".r - final val PureTimeRegex = """([+\-]?\d+)""".r + final val TimeRegex = """^([+\-]?\d+)(d|h|m|s|ms)$""".r + final val PureTimeRegex = """^([+\-]?\d+)$""".r def milliseconds(timeString: String): Option[Long] = { val value: Option[Long] = { diff --git a/measure/src/test/resources/config-test.json b/measure/src/test/resources/config-test.json new file mode 100644 index 000000000..8172038b6 --- /dev/null +++ b/measure/src/test/resources/config-test.json @@ -0,0 +1,52 @@ +{ + "name": "accu batch test", + + "process.type": "batch", + + "data.sources": [ + { + "name": "source", + "connectors": [ + { + "type": "avro", + "version": "1.7", + "config": { + "file.name": "src/test/resources/users_info_src.avro" + } + } + ] + }, { + "name": "target", + "connectors": [ + { + "type": "avro", + "version": "1.7", + "config": { + "file.name": "src/test/resources/users_info_target.avro" + } + } + ] + } + ], + + "evaluateRule": { + "rules": [ + { + "dsl.type": "spark-sql", + "name": "miss.record", + "rule": "SELECT source.user_id, source.first_name, source.last_name, source.address, source.email, source.phone, source.post_code FROM source LEFT JOIN target ON coalesce(source.user_id, 'null') = coalesce(target.user_id, 'null') AND coalesce(source.first_name, 'null') = coalesce(target.first_name, 'null') AND coalesce(source.last_name, 'null') = coalesce(target.last_name, 'null') AND coalesce(source.address, 'null') = coalesce(target.address, 'null') AND coalesce(source.email, 'null') = coalesce(target.email, 'null') AND coalesce(source.phone, 'null') = coalesce(target.phone, 'null') AND coalesce(source.post_code, 'null') = coalesce(target.post_code, 'null') WHERE (NOT (source.user_id IS NULL AND source.first_name IS NULL AND source.last_name IS NULL AND source.address IS NULL AND source.email IS NULL AND source.phone IS NULL AND source.post_code IS NULL)) AND (target.user_id IS NULL AND target.first_name IS NULL AND target.last_name IS NULL AND target.address IS NULL AND target.email IS NULL AND target.phone IS NULL AND target.post_code IS NULL)", + "persist.type": "record" + }, { + "dsl.type": "spark-sql", + "name": "miss.count", + "rule": "SELECT COUNT(*) AS `miss.count` FROM `miss.record`", + "persist.type": "metric" + }, { + "dsl.type": "spark-sql", + "name": "total.count", + "rule": "SELECT COUNT(*) AS `total.count` FROM source", + "persist.type": "metric" + } + ] + } +} \ No newline at end of file diff --git a/measure/src/test/resources/config-test1.json b/measure/src/test/resources/config-test1.json new file mode 100644 index 000000000..8459e1f4b --- /dev/null +++ b/measure/src/test/resources/config-test1.json @@ -0,0 +1,89 @@ +{ + "name": "accu batch test", + + "process.type": "batch", + + "data.sources": [ + { + "name": "source", + "connectors": [ + { + "type": "hive", + "version": "1.2", + "config": { + "database": "default", + "table.name": "src" + } + } + ] + }, { + "name": "target", + "connectors": [ + { + "type": "hive", + "version": "1.2", + "config": { + "database": "default", + "table.name": "tgt" + } + } + ] + } + ], + + "evaluateRule": { + "rules": [ + { + "dsl.type": "spark-opr", + "name": "source", + "rule": "from_json(source.value)" + }, + { + "dsl.type": "spark-sql", + "name": "seeds", + "rule": "SELECT explode(seeds) as seed FROM source" + }, + { + "dsl.type": "spark-opr", + "name": "seeds", + "rule": "from_json(seeds.seed)" + }, + { + "dsl.type": "spark-sql", + "name": "source", + "rule": "SELECT url, get_json_object(metadata, '$.tracker.crawlRequestCreateTS') AS ts FROM seeds" + }, + { + "dsl.type": "spark-opr", + "name": "target", + "rule": "from_json(target.value)" + }, + { + "dsl.type": "spark-sql", + "name": "attrs", + "rule": "SELECT groups[0].attrsList AS attrs FROM target" + }, + { + "dsl.type": "spark-sql", + "name": "target", + "rule": "SELECT attrs.values[index_of(attrs.name, 'URL')][0] AS url, get_json_object(attrs.values[index_of(attrs.name, 'CRAWLMETADATA')][0], '$.tracker.crawlRequestCreateTS') AS ts FROM df2" + }, + { + "dsl.type": "spark-sql", + "name": "miss.record", + "rule": "SELECT source.url, source.ts FROM source LEFT JOIN target ON coalesce(source.url, '') = coalesce(target.url, '') AND coalesce(source.ts, '') = coalesce(target.ts, '') WHERE (NOT (source.url IS NULL AND source.ts IS NULL)) AND (target.url IS NULL AND target.ts IS NULL)", + "persist.type": "record" + }, { + "dsl.type": "spark-sql", + "name": "miss.count", + "rule": "SELECT COUNT(*) AS `miss.count` FROM `miss.record`", + "persist.type": "metric" + }, { + "dsl.type": "spark-sql", + "name": "total.count", + "rule": "SELECT COUNT(*) AS `total.count` FROM source", + "persist.type": "metric" + } + ] + } +} \ No newline at end of file diff --git a/measure/src/test/resources/config-test2.json b/measure/src/test/resources/config-test2.json new file mode 100644 index 000000000..1a49d8b2a --- /dev/null +++ b/measure/src/test/resources/config-test2.json @@ -0,0 +1,57 @@ +{ + "name": "accu batch test", + + "process.type": "batch", + + "data.sources": [ + { + "name": "source", + "connectors": [ + { + "type": "avro", + "version": "1.7", + "config": { + "file.name": "src/test/resources/users_info_src.avro" + } + } + ] + }, { + "name": "target", + "connectors": [ + { + "type": "avro", + "version": "1.7", + "config": { + "file.name": "src/test/resources/users_info_target.avro" + } + } + ] + } + ], + + "evaluateRule": { + "rules": [ + { + "dsl.type": "griffin-dsl", + "dq.type": "accuracy", + "rule": "(source.user_id as id) = target.user_id AND source.first_name = 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", + "target": "target", + "miss.record": { + "name": "miss.record", + "persist.type": "record" + }, + "miss.count": { + "name": "miss.count", + "persist.type": "metric" + }, + "total.count": { + "name": "total.count", + "persist.type": "metric" + } + } + } + ] + } +} \ No newline at end of file diff --git a/measure/src/test/resources/env-test.json b/measure/src/test/resources/env-test.json new file mode 100644 index 000000000..603fad8a1 --- /dev/null +++ b/measure/src/test/resources/env-test.json @@ -0,0 +1,38 @@ +{ + "spark": { + "log.level": "WARN", + "checkpoint.dir": "hdfs:///griffin/batch/cp", + "batch.interval": "10s", + "process.interval": "10m", + "config": { + "spark.master": "local[*]" + } + }, + + "persist": [ + { + "type": "log", + "config": { + "max.log.lines": 100 + } + } + ], + + "info.cache": [ + { + "type": "zk", + "config": { + "hosts": "localhost:2181", + "namespace": "griffin/infocache", + "lock.path": "lock", + "mode": "persist", + "init.clear": true, + "close.clear": false + } + } + ], + + "cleaner": { + + } +} \ No newline at end of file diff --git a/measure/src/test/resources/input.msg b/measure/src/test/resources/input.msg new file mode 100644 index 000000000..edb4619e0 --- /dev/null +++ b/measure/src/test/resources/input.msg @@ -0,0 +1 @@ +{"confId":28,"seeds":["{\"url\":\"https://www.amazon.com/bridge-across-time-myths-history/dp/0855000449/ref\\u003dsr_1_1/186-6687480-5099813?ie\\u003dUTF8\\u0026keywords\\u003d0855000449\\u0026qid\\u003d1434663708\\u0026sr\\u003d8-1\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"0855000449\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/product/8479538562/ref\\u003dsr_1_9?ie\\u003dUTF8\\u0026qid\\u003d47073302\\u0026sr\\u003d8-1\\u0026keywords\\u003dnull\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"8479538562\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/product/1619619172/ref\\u003ds9_simh_gw_p63_d3_i3?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-1\\u0026pf_rd_r\\u003d80956L21ZP4Y9DRF74Z5\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d988263899\\u0026pf_rd_i\\u003d891862\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1619619172\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/1857751507/ref\\u003ds9_simh_gw_p74_d2_i8?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-2\\u0026pf_rd_r\\u003d81TU9DQLRW1TQM56LWW3\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d156743226\\u0026pf_rd_i\\u003d729862\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1857751507\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/product/1481714317/ref\\u003ds9_simh_gw_p107_d0_i4?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-1\\u0026pf_rd_r\\u003dKB37RY3JE5HKQ5G4630T\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d331346081\\u0026pf_rd_i\\u003d575468\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1481714317\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/product/1403775141/ref\\u003dcm_cr_pr_product_top\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1403775141\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Letters-Amelia-Diana-Turner-Forte/dp/0533157684/ref\\u003dsr_1_1/184-7208233-4184259?ie\\u003dUTF8\\u0026keywords\\u003d9780533157686\\u0026qid\\u003d1434675492\\u0026sr\\u003d8-1\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"0533157684\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/product/1591022320/ref\\u003ds9_simh_gw_p63_d3_i5?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-2\\u0026pf_rd_r\\u003dWVP9OE92HD77NSJXQZIL\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d556903239\\u0026pf_rd_i\\u003d594333\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1591022320\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Medieval-Early-Modern-Times-Janet/dp/9994609912/ref\\u003dsr_1_1/191-6299628-5905209?ie\\u003dUTF8\\u0026keywords\\u003d9789994609918\\u0026qid\\u003d1434432677\\u0026sr\\u003d8-1\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"9994609912\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/gp/product/1493574175/ref\\u003ds9_simh_gw_p74_d2_i6?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-3\\u0026pf_rd_r\\u003d1OZ6A0RNM4S8JJ01245S\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d124388178\\u0026pf_rd_i\\u003d596200\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1493574175\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/1598598635/ref\\u003ds9_ri_gw_g201_i7?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-3\\u0026pf_rd_r\\u003d7O715KMM6Y744QTQ4LIU\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d673955261\\u0026pf_rd_i\\u003d649226\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1598598635\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Escape-Tibet-Nick-Gray/dp/095715190X/ref\\u003dsr_1_1/175-7416659-0137139?ie\\u003dUTF8\\u0026keywords\\u003d9780957151901\\u0026qid\\u003d1434508924\\u0026sr\\u003d8-1\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"095715190X\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Man-About-House-Definitive-Companion-ebook/dp/B005AKCFXA\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"B005AKCFXA\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/gp/product/112067641X/ref\\u003ds9_ri_gw_g201_i5?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-3\\u0026pf_rd_r\\u003d0SIZV93I9RQ1669EES6L\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d504839570\\u0026pf_rd_i\\u003d161781\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"112067641X\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Unnatural-Death-Pedigree-Dorothy-Sayers/dp/B005TCM1X8/ref\\u003dsr_1_1/188-6111852-5942804?ie\\u003dUTF8\\u0026keywords\\u003d9780450020988\\u0026qid\\u003d1434450513\\u0026sr\\u003d8-1\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"B005TCM1X8\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/James-Castle-Common-Place-0970425716/dp/B002J813MA/ref\\u003dsr_1_2/176-6368479-5598662?ie\\u003dUTF8\\u0026keywords\\u003d0970425716\\u0026qid\\u003d1434422520\\u0026sr\\u003d8-2\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"B002J813MA\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/1940516099/ref\\u003ds9_ri_gw_g201_i2?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-1\\u0026pf_rd_r\\u003dR8JQ4JMW6P3SOJL36M9M\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d535308337\\u0026pf_rd_i\\u003d184331\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1940516099\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/product/1240463456/ref\\u003ds9_ri_gw_g201_i3?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-3\\u0026pf_rd_r\\u003dBL66W75USY907YG001QF\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d720488364\\u0026pf_rd_i\\u003d676890\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1240463456\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/CARSON-DELLOSA-BRIDGE-ORANGE-4-5/dp/1932210652/ref\\u003dsr_1_1/186-9625969-7817661?ie\\u003dUTF8\\u0026keywords\\u003d1932210652\\u0026qid\\u003d1434486504\\u0026sr\\u003d8-1\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1932210652\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/product/B00898M8X2/ref\\u003ds9_simh_gw_p74_d2_i6?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-2\\u0026pf_rd_r\\u003d6RTM9ED4621X306QR2Y3\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d205846301\\u0026pf_rd_i\\u003d110484\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"B00898M8X2\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/product/1597549045/ref\\u003ds9_simh_gw_p63_d3_i10?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-2\\u0026pf_rd_r\\u003dOLWI416TK382276FRI8S\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d971991139\\u0026pf_rd_i\\u003d747799\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1597549045\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Betrayal-Empty-Coffin-Novel/dp/1469216868/ref\\u003dsr_1_1/175-7820384-1782744?ie\\u003dUTF8\\u0026keywords\\u003d1469216868\\u0026qid\\u003d1434594698\\u0026sr\\u003d8-1\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1469216868\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/American-Police-Systems-Raymond-Fosdick/dp/B001KUVGPW/ref\\u003dsr_1_1/176-3370848-7400366?ie\\u003dUTF8\\u0026keywords\\u003d0875859097\\u0026qid\\u003d1434597754\\u0026sr\\u003d8-1\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"B001KUVGPW\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/gp/product/B00KDV1514/ref\\u003dsr_1_1?ie\\u003dUTF8\\u0026qid\\u003d68571520\\u0026sr\\u003d8-1\\u0026keywords\\u003dnull\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"B00KDV1514\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/product/1599928825/ref\\u003ds9_ri_gw_g201_i7?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-1\\u0026pf_rd_r\\u003d4TLP90H57YN3J9USNZS0\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d206214160\\u0026pf_rd_i\\u003d410077\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1599928825\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Obeah-Bible-L-W-Laurence/dp/1456472992/184-5031071-1689052?ie\\u003dUTF8\\u0026ref_\\u003ds9_simh_gw_p107_d0_i9\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1456472992\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/product/B00XJOGUL2/ref\\u003ds9_simh_gw_p63_d3_i4?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-1\\u0026pf_rd_r\\u003d6K4IS5NT23VSY8RK09SM\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d971785392\\u0026pf_rd_i\\u003d254059\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"metaCategId\\\":625,\\\"leafCategId\\\":43479,\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"B00XJOGUL2\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/product/0735841594/ref\\u003ds9_simh_gw_p74_d2_i2?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-3\\u0026pf_rd_r\\u003d4S47E7MUXUTX2OAS03M7\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d858738605\\u0026pf_rd_i\\u003d329711\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"0735841594\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/gp/product/1401246125/ref\\u003ds9_ri_gw_g201_i1?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-2\\u0026pf_rd_r\\u003dD5ZSN8FGJFRE1P6P06FN\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d986128032\\u0026pf_rd_i\\u003d431901\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1401246125\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/gp/product/0531233545/ref\\u003dcm_cr_pr_product_top\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"0531233545\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Magpies-Psychological-Thriller-Mark-Edwards/dp/1483911896/ref\\u003dsr_1_1/186-3277371-2912266?ie\\u003dUTF8\\u0026keywords\\u003d9781483911892\\u0026qid\\u003d1434440841\\u0026sr\\u003d8-1\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1483911896\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/gp/product/149936816X/ref\\u003ds9_ri_gw_g201_i9?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-3\\u0026pf_rd_r\\u003dA2G0NAMUI8Y2SLIOG6K1\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d206377910\\u0026pf_rd_i\\u003d192380\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"149936816X\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/gp/product/0321993306/ref\\u003ds9_simh_gw_p74_d2_i9?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-2\\u0026pf_rd_r\\u003dP49OLGZR6428DSLYD41K\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d464469450\\u0026pf_rd_i\\u003d713302\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"0321993306\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Arcana-Archives-Catalogo-Febbraio-9-Settembre/dp/8860604222/ref\\u003dsr_1_1/191-2294536-5098349?ie\\u003dUTF8\\u0026keywords\\u003d9788860604224\\u0026qid\\u003d1434516786\\u0026sr\\u003d8-1\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"8860604222\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/1287198279/ref\\u003dcm_cr_pr_product_top\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1287198279\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/gp/product/B005AW9VPI/ref\\u003ds9_simh_gw_p79_d17_i9?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-1\\u0026pf_rd_r\\u003dSH241RMHIXZ0P4OG3QO7\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d946293479\\u0026pf_rd_i\\u003d418397\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"B005AW9VPI\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/product/1844651258/ref\\u003dsr_1_2?ie\\u003dUTF8\\u0026qid\\u003d70179092\\u0026sr\\u003d8-1\\u0026keywords\\u003dnull\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1844651258\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/1484105176/ref\\u003dsr_1_4?ie\\u003dUTF8\\u0026qid\\u003d40145443\\u0026sr\\u003d8-1\\u0026keywords\\u003dnull\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1484105176\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Monday-Morning-Leadership-Valerie-Sokolosky/dp/097464031X/ref\\u003dsr_1_1/192-6857494-6582456?ie\\u003dUTF8\\u0026keywords\\u003d9780974640310\\u0026qid\\u003d1434634974\\u0026sr\\u003d8-1\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"097464031X\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/1288670427/ref\\u003ds9_simh_gw_p79_d17_i5?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-2\\u0026pf_rd_r\\u003dEF7S9AY28SM61TL6P5XR\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d377137900\\u0026pf_rd_i\\u003d382960\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1288670427\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/B00K8UNGGW/ref\\u003dcm_cr_pr_product_top\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"B00K8UNGGW\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Watcher-Another-World-J-Wilson/dp/187167672X/ref\\u003dsr_1_1/179-3023112-0477816?ie\\u003dUTF8\\u0026keywords\\u003d9781871676723\\u0026qid\\u003d1434542107\\u0026sr\\u003d8-1\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"187167672X\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Mad-Black-Lady-Wanda-Coleman/dp/0876854129/ref\\u003dsr_1_1/182-7097983-9105503?ie\\u003dUTF8\\u0026keywords\\u003d0876854129\\u0026qid\\u003d1434701480\\u0026sr\\u003d8-1\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"0876854129\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Every-Word-Fist-Amelia-Garcia/dp/146620818X\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"146620818X\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/product/B00873B0CO/ref\\u003ds9_simh_gw_p74_d2_i5?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-2\\u0026pf_rd_r\\u003d0XY59A109S39P6ID1N23\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d295494213\\u0026pf_rd_i\\u003d478561\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"B00873B0CO\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Springboard-Discovery-Mary-Lou-Lacy/dp/0804235953\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"0804235953\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Aventures-extraordinaires-dAdÃ%C2%83Â%C2%83Â%C2%83Ã%C2%83Â%C2%82Â%C2%83Ã%C2%83Â%C2%82Â%C2%83Ã%C2%83Â%C2%83Â%C2%83Ã%C2%83Â%C2%82Â%C2%82Ã%C2%83Â%C2%82Â%C2%83Ã%C2%83Â%C2%83Â%C2%83Ã%C2%83Â%C2%82Â%C2%83Ã%C2%83Â%C2%82Â%C2%85Ã%C2%83Â%C2%83Â%C2%83Ã%C2%83Â%C2%82Â%C2%82Ã%C2%83Â%C2%83Â%C2%82Ã%C2%83Â%C2%82Ã%C2%82¡le-Blanc-Sec/dp/229032096X\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"229032096X\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/product/1480512796/ref\\u003ds9_ri_gw_g201_i4?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-3\\u0026pf_rd_r\\u003dK4N7A64LBYP7TXSR9W49\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d681535512\\u0026pf_rd_i\\u003d230294\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1480512796\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/crise-conscience-europeenne-1680-1715-French/dp/221300613X/ref\\u003dsr_1_1/189-4114609-4176061?ie\\u003dUTF8\\u0026keywords\\u003d9782213006130\\u0026qid\\u003d1434473448\\u0026sr\\u003d8-1\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"221300613X\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/1743170246/ref\\u003dcm_cr_pr_product_top\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1743170246\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/1249925037/ref\\u003ds9_simh_gw_p63_d3_i2?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-3\\u0026pf_rd_r\\u003d1ZX7BOB71HEJOS512320\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d213504420\\u0026pf_rd_i\\u003d514621\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1249925037\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Lifetime-Volunteer-Frates-Joan-Gilmore/dp/1885596499/ref\\u003dsr_1_1/184-6208647-5211900?ie\\u003dUTF8\\u0026keywords\\u003d1885596499\\u0026qid\\u003d1434664139\\u0026sr\\u003d8-1\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1885596499\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/product/1507597142/ref\\u003ds9_simh_gw_p107_d0_i8?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-1\\u0026pf_rd_r\\u003dG88Q8FMLD0RS001F6STG\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d192898073\\u0026pf_rd_i\\u003d725640\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1507597142\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/gp/product/0691614385/ref\\u003dsr_1_7?ie\\u003dUTF8\\u0026qid\\u003d11874892\\u0026sr\\u003d8-1\\u0026keywords\\u003dnull\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"0691614385\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/gp/product/150531089X/ref\\u003dsr_1_8?ie\\u003dUTF8\\u0026qid\\u003d93416624\\u0026sr\\u003d8-1\\u0026keywords\\u003dnull\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"150531089X\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/gp/product/1591859581/ref\\u003ds9_simh_gw_p74_d2_i7?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-1\\u0026pf_rd_r\\u003dD89JO942AP0BGI8VHY6Z\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d259933109\\u0026pf_rd_i\\u003d594179\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1591859581\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/product/B0077D8O60/ref\\u003dcm_cr_pr_product_top\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"B0077D8O60\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/gp/product/0957401515/ref\\u003ds9_simh_gw_p63_d3_i2?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-3\\u0026pf_rd_r\\u003dP68SFUUGN08H8EL54714\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d948202552\\u0026pf_rd_i\\u003d899110\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"0957401515\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/3849554147/ref\\u003ds9_simh_gw_p79_d17_i8?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-2\\u0026pf_rd_r\\u003d1QMUT47S5N1OP87AM79G\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d664853631\\u0026pf_rd_i\\u003d825108\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"3849554147\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Regulating-Pesticides-Commission-Natural-Resources/dp/0309029465/ref\\u003dsr_1_1/186-2886194-1044035?ie\\u003dUTF8\\u0026keywords\\u003d0309029465\\u0026qid\\u003d1434700978\\u0026sr\\u003d8-1\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"0309029465\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/AutoCAD-2006-Tutorial-First-Fundamentals/dp/1585032301\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1585032301\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/product/1493162667/ref\\u003ds9_simh_gw_p63_d3_i8?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-1\\u0026pf_rd_r\\u003d5B1N58OK9DGNSXAYO648\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d490117977\\u0026pf_rd_i\\u003d302914\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1493162667\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/gp/product/1482532700/ref\\u003ds9_simh_gw_p79_d17_i4?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-1\\u0026pf_rd_r\\u003d2A85UB174G4QB67Q6W4E\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d469121724\\u0026pf_rd_i\\u003d652299\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1482532700\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/product/1289164118/ref\\u003dsr_1_2?ie\\u003dUTF8\\u0026qid\\u003d18547913\\u0026sr\\u003d8-1\\u0026keywords\\u003dnull\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1289164118\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/1275841597/ref\\u003dsr_1_6?ie\\u003dUTF8\\u0026qid\\u003d83157364\\u0026sr\\u003d8-1\\u0026keywords\\u003dnull\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1275841597\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/How-Maximize-Your-Profit-maximize/dp/1505906075\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1505906075\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/1436679060/ref\\u003ds9_simh_gw_p79_d17_i1?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-2\\u0026pf_rd_r\\u003dJ9SOI248X12J3TS2DL0V\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d526717715\\u0026pf_rd_i\\u003d369502\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1436679060\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Plant-Molecular-Biology-Essential-Techniques/dp/0471972681/ref\\u003dsr_1_1/190-9297182-6303650?ie\\u003dUTF8\\u0026keywords\\u003d9780471972686\\u0026qid\\u003d1434705995\\u0026sr\\u003d8-1\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"0471972681\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Europe-Exporters-Handbook-Jenner/dp/0871965658/ref\\u003dsr_1_1/189-3435491-9141552?ie\\u003dUTF8\\u0026keywords\\u003d9780871965653\\u0026qid\\u003d1434701852\\u0026sr\\u003d8-1\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"0871965658\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/product/1288527004/ref\\u003ds9_simh_gw_p79_d17_i2?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-1\\u0026pf_rd_r\\u003dL676IU8SE79EB28460T4\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d839567868\\u0026pf_rd_i\\u003d952718\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1288527004\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Groups-Practice-Marianne-Schneider-Hardcover/dp/0534367453/ref\\u003dsr_1_1/186-0267917-7112209?ie\\u003dUTF8\\u0026keywords\\u003d0534367453\\u0026qid\\u003d1434879089\\u0026sr\\u003d8-1\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"0534367453\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/product/161034958X/ref\\u003ds9_simh_gw_p79_d17_i3?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-2\\u0026pf_rd_r\\u003dLDIXFU564R4V24ZK6SN2\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d473453365\\u0026pf_rd_i\\u003d771807\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"161034958X\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/product/B00049QL5U/ref\\u003ds9_simh_gw_p107_d0_i2?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-2\\u0026pf_rd_r\\u003dY564P8EU9SRSEXNYP77G\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d561884715\\u0026pf_rd_i\\u003d312289\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"B00049QL5U\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/product/1618852183/ref\\u003ds9_simh_gw_p107_d0_i8?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-3\\u0026pf_rd_r\\u003dKM0Q6544JP28U5676KZF\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d974311606\\u0026pf_rd_i\\u003d652689\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1618852183\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Models-Dermatology-Vol/dp/3805547617\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"3805547617\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/gp/product/0684859726/ref\\u003ds9_simh_gw_p79_d17_i7?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-2\\u0026pf_rd_r\\u003d727ES4S50NPD7M8F4XLO\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d475467204\\u0026pf_rd_i\\u003d897737\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"0684859726\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/gp/product/1449788386/ref\\u003dsr_1_3?ie\\u003dUTF8\\u0026qid\\u003d34593636\\u0026sr\\u003d8-1\\u0026keywords\\u003dnull\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1449788386\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/1622125290/ref\\u003ds9_simh_gw_p107_d0_i7?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-1\\u0026pf_rd_r\\u003d46Q876OOMM3D6XSDG7VM\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d954847296\\u0026pf_rd_i\\u003d374067\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1622125290\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/gp/product/1494559447/ref\\u003ds9_simh_gw_p79_d17_i8?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-1\\u0026pf_rd_r\\u003dW5LSQ3KS109584JUY4A0\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d694179817\\u0026pf_rd_i\\u003d134867\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1494559447\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Proceedings-1-2-Classical-Association-Wales/dp/1236636929/ref\\u003dsr_1_1/184-1359550-5736018?ie\\u003dUTF8\\u0026keywords\\u003d9781236636928\\u0026qid\\u003d1434649299\\u0026sr\\u003d8-1\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1236636929\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/gp/product/1287003001/ref\\u003dcm_cr_pr_product_top\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1287003001\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/gp/product/1249180309/ref\\u003dsr_1_10?ie\\u003dUTF8\\u0026qid\\u003d32633819\\u0026sr\\u003d8-1\\u0026keywords\\u003dnull\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1249180309\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Michelin-Red-Guide-Spain-Portugal/dp/2060063779/ref\\u003dsr_1_1/179-4403527-2002065?ie\\u003dUTF8\\u0026keywords\\u003d9782060063775\\u0026qid\\u003d1434621406\\u0026sr\\u003d8-1\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"2060063779\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Homicidal-Aliens-Other-Disappointments-Invasion/dp/1480518522/ref\\u003dsr_1_1/186-9648743-6858653?ie\\u003dUTF8\\u0026keywords\\u003d1480518522\\u0026qid\\u003d1434660937\\u0026sr\\u003d8-1\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1480518522\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/0984635742/ref\\u003ds9_simh_gw_p79_d17_i4?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-1\\u0026pf_rd_r\\u003d21YWJSFV761872B02I7H\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d175584267\\u0026pf_rd_i\\u003d665215\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"0984635742\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/product/1583335331/ref\\u003ds9_simh_gw_p74_d2_i8?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-1\\u0026pf_rd_r\\u003dPLKAEVA1UJH99O85N5K2\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d919768077\\u0026pf_rd_i\\u003d142182\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1583335331\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/1305256662/ref\\u003ds9_ri_gw_g201_i6?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-2\\u0026pf_rd_r\\u003d8V35F53XPBW54M0TO89S\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d736378767\\u0026pf_rd_i\\u003d803718\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1305256662\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Biological-Rhythm-Research-Sollberger/dp/0444405496/ref\\u003dsr_1_1/184-0842412-0577011?ie\\u003dUTF8\\u0026keywords\\u003d0444405496\\u0026qid\\u003d1434660594\\u0026sr\\u003d8-1\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"0444405496\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/product/0826520235/ref\\u003ds9_simh_gw_p79_d17_i8?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-2\\u0026pf_rd_r\\u003d17N0U1085ISNRS396YHI\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d909993468\\u0026pf_rd_i\\u003d167394\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"0826520235\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/1452089175/ref\\u003ds9_simh_gw_p63_d3_i7?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-2\\u0026pf_rd_r\\u003d9S13UGLB1J3M03X087L7\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d683610611\\u0026pf_rd_i\\u003d897880\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1452089175\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/gp/product/1286720451/ref\\u003ds9_simh_gw_p79_d17_i2?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-2\\u0026pf_rd_r\\u003dE82MQGB807FFVWB3Q44R\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d104104751\\u0026pf_rd_i\\u003d243022\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1286720451\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/product/1499041055/ref\\u003ds9_ri_gw_g201_i9?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-1\\u0026pf_rd_r\\u003dB5S17N4WLLSPDD34YDY8\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d872947608\\u0026pf_rd_i\\u003d644957\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1499041055\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/1288769598/ref\\u003dcm_cr_pr_product_top\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1288769598\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Encyclopaedia-United-States-Spacecraft-Bison/dp/0600500519/ref\\u003dsr_1_1/189-9370660-2213859?ie\\u003dUTF8\\u0026keywords\\u003d9780600500513\\u0026qid\\u003d1434620291\\u0026sr\\u003d8-1\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"0600500519\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/ANTONIO-VIVALDI-Garland-reference-humanities/dp/0824083865\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"0824083865\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/gp/product/1465417117/ref\\u003dsr_1_4?ie\\u003dUTF8\\u0026qid\\u003d16061560\\u0026sr\\u003d8-1\\u0026keywords\\u003dnull\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1465417117\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/gp/product/0813224535/ref\\u003ds9_simh_gw_p74_d2_i6?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-3\\u0026pf_rd_r\\u003dSHQ8F60N372GY56JWF09\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d412949045\\u0026pf_rd_i\\u003d425983\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"0813224535\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/gp/product/1495421996/ref\\u003ds9_ri_gw_g201_i4?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-2\\u0026pf_rd_r\\u003dX9D65SVNFOVU7N8SWF3O\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d165045927\\u0026pf_rd_i\\u003d372797\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1495421996\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/product/3640223098/ref\\u003dcm_cr_pr_product_top\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"3640223098\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Soldier-Official-Guide-Accompany-ITV/dp/B0041CQMJG\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"B0041CQMJG\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/1480031003/ref\\u003ds9_simh_gw_p63_d3_i10?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-2\\u0026pf_rd_r\\u003d0RSF4Y1580S2V22H2JSG\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d328419621\\u0026pf_rd_i\\u003d991785\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1480031003\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/My-Harvest-Home-Celebration-Customs/dp/0937203688/ref\\u003dsr_1_1/185-2501167-9365558?ie\\u003dUTF8\\u0026keywords\\u003d9780937203682\\u0026qid\\u003d1434501536\\u0026sr\\u003d8-1\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"0937203688\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Halfway-Decent-Ernie-Hudson/dp/B001A4YO20\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"B001A4YO20\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/1288916906/ref\\u003ds9_simh_gw_p74_d2_i6?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-3\\u0026pf_rd_r\\u003dONRG59HTI83U545RG01Y\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d669707968\\u0026pf_rd_i\\u003d353895\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1288916906\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/product/1288725108/ref\\u003dsr_1_5?ie\\u003dUTF8\\u0026qid\\u003d97736076\\u0026sr\\u003d8-1\\u0026keywords\\u003dnull\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1288725108\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/product/1462067514/ref\\u003dcm_cr_pr_product_top\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1462067514\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/1290713707/ref\\u003ds9_simh_gw_p107_d0_i8?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-1\\u0026pf_rd_r\\u003dR9AF8E0I0OVS17OZWIL0\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d819156518\\u0026pf_rd_i\\u003d124919\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1290713707\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Het-historische-gedicht-geschiedenis-Vlaanderen/dp/9057592851/ref\\u003dsr_1_1/176-4708134-2081319?ie\\u003dUTF8\\u0026keywords\\u003d9789057592850\\u0026qid\\u003d1434457720\\u0026sr\\u003d8-1\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"9057592851\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/product/1289300151/ref\\u003ds9_simh_gw_p63_d3_i3?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-3\\u0026pf_rd_r\\u003dDW5AQM8981BG14F3OR4U\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d353942470\\u0026pf_rd_i\\u003d642212\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1289300151\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Christines-Picture-Book-Christian-Andersen/dp/0862720893/ref\\u003dsr_1_1/179-3046203-1921448?ie\\u003dUTF8\\u0026keywords\\u003d9780862720896\\u0026qid\\u003d1434426244\\u0026sr\\u003d8-1\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"0862720893\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/product/1249461162/ref\\u003ds9_simh_gw_p79_d17_i3?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-1\\u0026pf_rd_r\\u003d57YUBLM3S5JH10JY6SO4\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d826066547\\u0026pf_rd_i\\u003d907699\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1249461162\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/1246822881/ref\\u003dsr_1_10?ie\\u003dUTF8\\u0026qid\\u003d18489479\\u0026sr\\u003d8-1\\u0026keywords\\u003dnull\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1246822881\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/product/1497903920/ref\\u003dcm_cr_pr_product_top\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1497903920\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/product/1105315800/ref\\u003ds9_simh_gw_p74_d2_i2?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-3\\u0026pf_rd_r\\u003dISLI9GZS5J520O513PK6\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d191025471\\u0026pf_rd_i\\u003d563906\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1105315800\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/gp/product/1249420024/ref\\u003ds9_simh_gw_p63_d3_i6?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-2\\u0026pf_rd_r\\u003dBG6V7MK62E7322D5VS4K\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d335458808\\u0026pf_rd_i\\u003d388332\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1249420024\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/product/1287198848/ref\\u003ds9_simh_gw_p107_d0_i10?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-3\\u0026pf_rd_r\\u003dUATP2780PEBSP3MZ85J9\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d724971579\\u0026pf_rd_i\\u003d737723\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1287198848\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/product/1599152088/ref\\u003ds9_simh_gw_p74_d2_i1?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-1\\u0026pf_rd_r\\u003d1R9QZBS3LHGN2QKTW84D\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d285986720\\u0026pf_rd_i\\u003d167207\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1599152088\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/product/1287183700/ref\\u003ds9_simh_gw_p79_d17_i7?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-3\\u0026pf_rd_r\\u003d4W49A4JP56S6IELDOF3S\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d604578060\\u0026pf_rd_i\\u003d666380\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1287183700\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/product/1470817403/ref\\u003ds9_simh_gw_p63_d3_i10?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-1\\u0026pf_rd_r\\u003d72AXH8EE5ZQTRY727NNV\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d213474611\\u0026pf_rd_i\\u003d351283\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1470817403\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Sonnets-Facing-Conviction-Christopher-Presfield/dp/1891812203\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1891812203\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Dead-Ringer-Jasmine-Cresswell/dp/B0010BSZ5U/ref\\u003dsr_1_1/190-6876153-5019461?ie\\u003dUTF8\\u0026keywords\\u003d1551667126\\u0026qid\\u003d1434458583\\u0026sr\\u003d8-1\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"B0010BSZ5U\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/1933660996/ref\\u003dcm_cr_pr_product_top\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1933660996\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Sherington-Fiefs-Fields-Buckinghamshire-Village/dp/0521046378/ref\\u003dsr_1_1/182-5625910-7791542?ie\\u003dUTF8\\u0026keywords\\u003d9780521046374\\u0026qid\\u003d1434591978\\u0026sr\\u003d8-1\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"0521046378\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/gp/product/0452297060/ref\\u003ds9_ri_gw_g201_i6?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-3\\u0026pf_rd_r\\u003d9WQLGU3K20TZ97OLNE9N\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d817830578\\u0026pf_rd_i\\u003d592054\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"0452297060\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Believers-Pocket-Companion-Needful-Sinners/dp/1889058092/ref\\u003dsr_1_1/181-8057383-5400821?ie\\u003dUTF8\\u0026keywords\\u003d1889058092\\u0026qid\\u003d1434591575\\u0026sr\\u003d8-1\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1889058092\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/gp/product/1288686684/ref\\u003ds9_simh_gw_p107_d0_i5?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-2\\u0026pf_rd_r\\u003d6G636UL89WPEJRNWM864\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d863114753\\u0026pf_rd_i\\u003d127597\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1288686684\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/product/1441968091/ref\\u003ds9_simh_gw_p107_d0_i7?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-1\\u0026pf_rd_r\\u003dVD3N8XQHIN218881396G\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d591946161\\u0026pf_rd_i\\u003d784879\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1441968091\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Illustrated-Skating-Dictionary-Young-People/dp/013451260X/ref\\u003dsr_1_1/187-7738327-4220041?ie\\u003dUTF8\\u0026keywords\\u003d9780134512600\\u0026qid\\u003d1434671356\\u0026sr\\u003d8-1\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"013451260X\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/gp/product/1286427355/ref\\u003ds9_simh_gw_p63_d3_i4?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-2\\u0026pf_rd_r\\u003dG08BDQF87W390SN0VKSY\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d609639660\\u0026pf_rd_i\\u003d873910\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1286427355\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/1592999107/ref\\u003ds9_simh_gw_p63_d3_i10?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-3\\u0026pf_rd_r\\u003d0GZEX2Z031943Y98PS2O\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d207358316\\u0026pf_rd_i\\u003d646461\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1592999107\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/product/0971487421/ref\\u003ds9_simh_gw_p79_d17_i4?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-3\\u0026pf_rd_r\\u003dVP2GYS4S48L27YR91NGX\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d288057837\\u0026pf_rd_i\\u003d811217\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"0971487421\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Dakine-Traction-12-25-Inch-Black-Smoke/dp/B009HYI5U2/ref\\u003dsr_1_1/190-3075409-4147065?ie\\u003dUTF8\\u0026keywords\\u003d0610934775921\\u0026qid\\u003d1434572027\\u0026sr\\u003d8-1\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"B009HYI5U2\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/gp/product/1505465214/ref\\u003ds9_simh_gw_p79_d17_i10?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-2\\u0026pf_rd_r\\u003dS88VYH0JSN0WV58N763S\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d469060850\\u0026pf_rd_i\\u003d923112\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1505465214\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/gp/product/1249477794/ref\\u003ds9_ri_gw_g201_i3?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-2\\u0026pf_rd_r\\u003d1YEA01A3X8LEV2A1XUYL\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d163201523\\u0026pf_rd_i\\u003d432311\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1249477794\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/gp/product/1439101515/ref\\u003dsr_1_10?ie\\u003dUTF8\\u0026qid\\u003d41353122\\u0026sr\\u003d8-1\\u0026keywords\\u003dnull\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1439101515\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/gp/product/0871522748/ref\\u003ds9_simh_gw_p63_d3_i8?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-3\\u0026pf_rd_r\\u003dM2XLTA5E84OZ9WW38KL1\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d173836115\\u0026pf_rd_i\\u003d585725\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"0871522748\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Guide-Florida-legal-research-Stupski/dp/0327155868/ref\\u003dsr_1_1/178-6731625-0968554?ie\\u003dUTF8\\u0026keywords\\u003d0327155868\\u0026qid\\u003d1434593715\\u0026sr\\u003d8-1\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"0327155868\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/product/0735341257/ref\\u003dcm_cr_pr_product_top\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"0735341257\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/1287251641/ref\\u003ds9_simh_gw_p63_d3_i1?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-1\\u0026pf_rd_r\\u003d1KEUJ6L7GYD56MD665L5\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d508856645\\u0026pf_rd_i\\u003d515955\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1287251641\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Points-2012-2013-Handbook-Brigade-Midshipmen/dp/161251149X/ref\\u003dsr_1_1/191-7390079-6376012?ie\\u003dUTF8\\u0026keywords\\u003d9781612511498\\u0026qid\\u003d1434621407\\u0026sr\\u003d8-1\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"161251149X\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/product/9050634788/ref\\u003dcm_cr_pr_product_top\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"9050634788\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/gp/product/1406829730/ref\\u003ds9_simh_gw_p63_d3_i8?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-3\\u0026pf_rd_r\\u003d86Y0VR3VTP14529B9TK2\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d516175225\\u0026pf_rd_i\\u003d152667\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1406829730\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/product/1120716756/ref\\u003dcm_cr_pr_product_top\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1120716756\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/gp/product/1449781837/ref\\u003ds9_ri_gw_g201_i9?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-2\\u0026pf_rd_r\\u003dQI0MOKJ0782RNRSXO3EJ\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d853693070\\u0026pf_rd_i\\u003d466575\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1449781837\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Omnibus-Pusher-Kings-Ransom-Money/dp/1407221035/ref\\u003dsr_1_1/187-4173899-5264528?ie\\u003dUTF8\\u0026keywords\\u003d9781407221038\\u0026qid\\u003d1434673964\\u0026sr\\u003d8-1\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1407221035\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/gp/product/1500892181/ref\\u003ds9_simh_gw_p107_d0_i1?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-3\\u0026pf_rd_r\\u003d3S5TF02AB6MRAFME7AKJ\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d572077303\\u0026pf_rd_i\\u003d484731\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1500892181\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/product/1847975313/ref\\u003ds9_ri_gw_g201_i9?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-3\\u0026pf_rd_r\\u003dAJO40FSP5P875338JO26\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d321291056\\u0026pf_rd_i\\u003d704134\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1847975313\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/1249000106/ref\\u003ds9_simh_gw_p74_d2_i4?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-2\\u0026pf_rd_r\\u003dIY3AAVY7WEF77SSKDY0S\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d124726089\\u0026pf_rd_i\\u003d902185\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1249000106\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Golden-Dawn-Journal-Book-Llewellyns/dp/1567188516\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1567188516\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Wee-Pals-Knowledge-Morrie-Turner/dp/0451058003\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"0451058003\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Sourcebook-Public-Record-Providers-Investigative/dp/1879792060\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1879792060\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/World-Omnibus-Books-Bertie-Webster/dp/0216883857/ref\\u003dsr_1_1/185-2474915-1241732?ie\\u003dUTF8\\u0026keywords\\u003d9780216883857\\u0026qid\\u003d1434671154\\u0026sr\\u003d8-1\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"0216883857\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/product/0252080238/ref\\u003ds9_simh_gw_p107_d0_i9?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-1\\u0026pf_rd_r\\u003dFGQSTXT42QRKQ4UL37BL\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d956630874\\u0026pf_rd_i\\u003d262703\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"0252080238\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Adjustment-Equity-Malaysia-Development-Studies/dp/9264136010/ref\\u003dsr_1_1/181-7587893-0591704?ie\\u003dUTF8\\u0026keywords\\u003d9264136010\\u0026qid\\u003d1434613893\\u0026sr\\u003d8-1\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"9264136010\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/product/1411672577/ref\\u003dsr_1_1?ie\\u003dUTF8\\u0026qid\\u003d63432192\\u0026sr\\u003d8-1\\u0026keywords\\u003dnull\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1411672577\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/product/3849566331/ref\\u003ds9_simh_gw_p74_d2_i6?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-1\\u0026pf_rd_r\\u003dSV6SV2689672R60MPXPM\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d618760996\\u0026pf_rd_i\\u003d491874\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"3849566331\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/product/1249325463/ref\\u003ds9_simh_gw_p74_d2_i7?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-1\\u0026pf_rd_r\\u003d3300UTE61DSF42Z24YR6\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d290302597\\u0026pf_rd_i\\u003d909049\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1249325463\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Through-Dark-Labyrinth-Biography-Lawrence/dp/0712666788/ref\\u003dsr_1_1/192-9529270-4371764?ie\\u003dUTF8\\u0026keywords\\u003d9780712666787\\u0026qid\\u003d1434616188\\u0026sr\\u003d8-1\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"0712666788\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/gp/product/127583633X/ref\\u003dcm_cr_pr_product_top\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"127583633X\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Maryland-Replacement-Business-Regulation---Business/dp/0327066628\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"0327066628\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Mysterious-Chills-Thrills-Laura-Hickey/dp/097490130X/ref\\u003dsr_1_1/178-6173788-1828330?ie\\u003dUTF8\\u0026keywords\\u003d9780974901305\\u0026qid\\u003d1434644270\\u0026sr\\u003d8-1\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"097490130X\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Danzig-American-Recordings-74321-23681/dp/B000026GYI/ref\\u003dsr_1_1/182-3883208-9254143?ie\\u003dUTF8\\u0026keywords\\u003d0743212368123\\u0026qid\\u003d1434534995\\u0026sr\\u003d8-1\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"B000026GYI\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/gp/product/1634284798/ref\\u003ds9_simh_gw_p107_d0_i6?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-2\\u0026pf_rd_r\\u003d3895AHWRBT66Z1P9QS0O\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d493371621\\u0026pf_rd_i\\u003d955857\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1634284798\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Auto-Ventshade-194719-Ventvisor-Deflector/dp/B000CFAV8O\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"B000CFAV8O\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/1936744279/ref\\u003ds9_ri_gw_g201_i1?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-1\\u0026pf_rd_r\\u003dINH4U6EIY9P1JLL2A096\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d691439560\\u0026pf_rd_i\\u003d121424\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1936744279\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/1433996014/ref\\u003ds9_simh_gw_p74_d2_i2?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-2\\u0026pf_rd_r\\u003d948GM1R0U4K60I8WI58U\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d727906654\\u0026pf_rd_i\\u003d590985\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1433996014\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Misterio-Metru-Mystery-Bionicle-Spanish/dp/8497632478/ref\\u003dsr_1_1/187-5126550-6996742?ie\\u003dUTF8\\u0026keywords\\u003d9788497632478\\u0026qid\\u003d1434450671\\u0026sr\\u003d8-1\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"8497632478\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/0941613453/ref\\u003dcm_cr_pr_product_top\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"0941613453\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/1480143405/ref\\u003dsr_1_9?ie\\u003dUTF8\\u0026qid\\u003d3814688\\u0026sr\\u003d8-1\\u0026keywords\\u003dnull\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1480143405\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/gp/product/B00GS339YC/ref\\u003ds9_simh_gw_p74_d2_i1?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-3\\u0026pf_rd_r\\u003d1H1S8S7P7AYN485A74B8\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d544362868\\u0026pf_rd_i\\u003d230478\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"metaCategId\\\":11700,\\\"leafCategId\\\":79630,\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"B00GS339YC\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/product/B005T820VK/ref\\u003ds9_simh_gw_p74_d2_i7?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-3\\u0026pf_rd_r\\u003dF3M98S0T37EEOS86X0SR\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d477234694\\u0026pf_rd_i\\u003d588761\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"metaCategId\\\":12576,\\\"leafCategId\\\":181877,\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"B005T820VK\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/gp/product/B00LMKZ5PM/ref\\u003dcm_cr_pr_product_top\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"metaCategId\\\":99,\\\"leafCategId\\\":166830,\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"B00LMKZ5PM\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Microsoft-Excel-Quick-Source-Guide/dp/1932104119/ref\\u003dsr_1_1/179-5887599-8112940?ie\\u003dUTF8\\u0026keywords\\u003d1932104119\\u0026qid\\u003d1434663197\\u0026sr\\u003d8-1\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1932104119\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/product/B00NODYOP2/ref\\u003ds9_simh_gw_p79_d17_i2?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-2\\u0026pf_rd_r\\u003dD1UKURZU7T0RSZDL9Z08\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d102955252\\u0026pf_rd_i\\u003d264326\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"B00NODYOP2\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Forests-What-Are-Andy-Owen/dp/0431023689/ref\\u003dsr_1_1/175-1909095-8188046?ie\\u003dUTF8\\u0026keywords\\u003d9780431023687\\u0026qid\\u003d1434516840\\u0026sr\\u003d8-1\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"0431023689\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Hannahs-brave-year-Rhoda-Wooldridge/dp/083090333X\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"083090333X\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Chemistry-Dummies-Turtleback-Library-Prebound/dp/061391550X/ref\\u003dsr_1_1/182-3704276-6074404?ie\\u003dUTF8\\u0026keywords\\u003d9780613915502\\u0026qid\\u003d1434684306\\u0026sr\\u003d8-1\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"061391550X\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Immigration-secrets-through-immigration-more--/dp/0964001314\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"0964001314\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Adapted-Tests-Prentice-Science-Explorer/dp/0131665979/ref\\u003dsr_1_1/188-0786267-7923645?ie\\u003dUTF8\\u0026keywords\\u003d0131665979\\u0026qid\\u003d1434704033\\u0026sr\\u003d8-1\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"0131665979\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Expense-Control-Productivity-Retailing-profit/dp/0867305436/ref\\u003dsr_1_1/191-1454825-6348502?ie\\u003dUTF8\\u0026keywords\\u003d0867305436\\u0026qid\\u003d1434627013\\u0026sr\\u003d8-1\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"0867305436\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/1455528854/ref\\u003dsr_1_6?ie\\u003dUTF8\\u0026qid\\u003d62509797\\u0026sr\\u003d8-1\\u0026keywords\\u003dnull\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1455528854\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/product/0987935801/ref\\u003dsr_1_6?ie\\u003dUTF8\\u0026qid\\u003d64553029\\u0026sr\\u003d8-1\\u0026keywords\\u003dnull\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"0987935801\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Lucy-Montgomery-Short-Stories-1905-1906/dp/1781392420/ref\\u003dsr_1_1/179-8969282-1218322?ie\\u003dUTF8\\u0026keywords\\u003d1781392420\\u0026qid\\u003d1434444399\\u0026sr\\u003d8-1\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1781392420\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/product/B00DV5WTX2/ref\\u003ds9_simh_gw_p79_d17_i4?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-2\\u0026pf_rd_r\\u003dD6W0KS18E69NEYB18001\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d815214806\\u0026pf_rd_i\\u003d447967\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"B00DV5WTX2\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/product/1461377293/ref\\u003ds9_ri_gw_g201_i10?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-3\\u0026pf_rd_r\\u003d030864T4Q4066704UZ7E\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d592325671\\u0026pf_rd_i\\u003d959581\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1461377293\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/product/1287021905/ref\\u003ds9_simh_gw_p63_d3_i9?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-1\\u0026pf_rd_r\\u003d4HXDWF45T32MD1D3S8GE\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d385879740\\u0026pf_rd_i\\u003d129580\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1287021905\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/1141127326/ref\\u003ds9_simh_gw_p63_d3_i2?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-2\\u0026pf_rd_r\\u003dBM8080B46NGFUSXBNDE1\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d872809181\\u0026pf_rd_i\\u003d473326\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1141127326\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/gp/product/1481755307/ref\\u003ds9_ri_gw_g201_i4?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-1\\u0026pf_rd_r\\u003d5ROK0FPJ6HH04V0O637L\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d115978925\\u0026pf_rd_i\\u003d569414\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1481755307\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/gp/product/1249332729/ref\\u003ds9_simh_gw_p107_d0_i9?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-1\\u0026pf_rd_r\\u003dVEX309MH2YPAS2X5SX95\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d373201442\\u0026pf_rd_i\\u003d192379\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1249332729\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/gp/product/B000FVQLY8/ref\\u003ds9_simh_gw_p79_d17_i4?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-3\\u0026pf_rd_r\\u003dZSVSV5L0Z5SG5D2ZF9IS\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d168856631\\u0026pf_rd_i\\u003d247360\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"B000FVQLY8\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/gp/product/1484914317/ref\\u003ds9_simh_gw_p74_d2_i4?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-3\\u0026pf_rd_r\\u003dGXDNY3K3KAPF1S08AYRO\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d775700064\\u0026pf_rd_i\\u003d704233\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1484914317\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/The-Wild-Dogs/dp/0947072535\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"0947072535\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/product/1489558047/ref\\u003ds9_ri_gw_g201_i6?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-1\\u0026pf_rd_r\\u003dQHWMSIYZ99493U8IN9E5\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d192725230\\u0026pf_rd_i\\u003d993622\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1489558047\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/product/0866906339/ref\\u003ds9_simh_gw_p107_d0_i4?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-2\\u0026pf_rd_r\\u003d7G9HE2IF43JWM74SZY2Z\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d930027883\\u0026pf_rd_i\\u003d439608\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"0866906339\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Wedding-Book-Experts-Planning-Day-Your-ebook/dp/B006MGDJB8\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"B006MGDJB8\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Dios-Habla-Hoy-Spanish-Edition/dp/1585161187/ref\\u003dsr_1_1/176-1062857-7907332?ie\\u003dUTF8\\u0026keywords\\u003d9781585161188\\u0026qid\\u003d1434437626\\u0026sr\\u003d8-1\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1585161187\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/product/551859366X/ref\\u003ds9_simh_gw_p63_d3_i8?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-2\\u0026pf_rd_r\\u003d7TW10D9B9W7501800JDZ\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d239708876\\u0026pf_rd_i\\u003d845749\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"551859366X\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/gp/product/B000TK7OZA/ref\\u003dsr_1_3?ie\\u003dUTF8\\u0026qid\\u003d41558171\\u0026sr\\u003d8-1\\u0026keywords\\u003dnull\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"B000TK7OZA\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/product/B00C77DS58/ref\\u003dsr_1_2?ie\\u003dUTF8\\u0026qid\\u003d70641569\\u0026sr\\u003d8-1\\u0026keywords\\u003dnull\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"metaCategId\\\":11700,\\\"leafCategId\\\":79654,\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"B00C77DS58\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/product/1499698259/ref\\u003ds9_simh_gw_p63_d3_i4?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-2\\u0026pf_rd_r\\u003d66W4059MU38M8JZ3RFA3\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d417187228\\u0026pf_rd_i\\u003d899409\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1499698259\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Souvenir-Guide-Fourth-Botanic-Gardens/dp/1842464140\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1842464140\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Beasts-Gor-John-Norman/dp/B001AP3KH4/ref\\u003dsr_1_1/190-0932607-6814544?ie\\u003dUTF8\\u0026keywords\\u003d0352303964\\u0026qid\\u003d1434627148\\u0026sr\\u003d8-1\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"B001AP3KH4\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Off-Balance-Purpose-Embrace-Uncertainty/dp/1608320146/ref\\u003dcm_cr_pr_product_top\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1608320146\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Metal-Vapor-Lasers-Applications-Proceedings/dp/0819405027/ref\\u003dsr_1_1/187-8627002-9363742?ie\\u003dUTF8\\u0026keywords\\u003d9780819405029\\u0026qid\\u003d1434697125\\u0026sr\\u003d8-1\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"0819405027\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Mormon-Temple-Square-Behind-Scenery/dp/0887147585/ref\\u003dsr_1_1/179-6133906-6553235?ie\\u003dUTF8\\u0026keywords\\u003d9780887147586\\u0026qid\\u003d1434458335\\u0026sr\\u003d8-1\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"0887147585\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/product/1249331412/ref\\u003ds9_simh_gw_p79_d17_i7?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-1\\u0026pf_rd_r\\u003dFZYF6X720561ZOQ6Q42O\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d730531613\\u0026pf_rd_i\\u003d644545\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1249331412\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Hablar-Idioma-Cliente-Healthcare-Professionals/dp/0966022572/ref\\u003dsr_1_1/180-5338436-6086643?ie\\u003dUTF8\\u0026keywords\\u003d0966022572\\u0026qid\\u003d1434505188\\u0026sr\\u003d8-1\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"0966022572\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Spoils-War-Destroyer-Warren-Murphy/dp/0759252343/ref\\u003dsr_1_1/188-0805395-8182724?ie\\u003dUTF8\\u0026keywords\\u003d9780759252349\\u0026qid\\u003d1434508721\\u0026sr\\u003d8-1\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"0759252343\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/gp/product/1619044161/ref\\u003ds9_simh_gw_p63_d3_i4?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-1\\u0026pf_rd_r\\u003d9LG2PTD6QGEZHREY432U\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d534380538\\u0026pf_rd_i\\u003d500944\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1619044161\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/gp/product/1277240469/ref\\u003ds9_ri_gw_g201_i9?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-2\\u0026pf_rd_r\\u003dOKXD96G4LIO1M5DK7256\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d747902558\\u0026pf_rd_i\\u003d884179\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1277240469\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/product/1277760233/ref\\u003ds9_simh_gw_p63_d3_i3?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-2\\u0026pf_rd_r\\u003d192K1XBMKJWZU4U9T670\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d553226685\\u0026pf_rd_i\\u003d556820\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1277760233\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/gp/product/1288354339/ref\\u003ds9_simh_gw_p107_d0_i4?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-3\\u0026pf_rd_r\\u003dSO55W703EIKF63LSATHH\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d576464099\\u0026pf_rd_i\\u003d961684\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1288354339\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/gp/product/B000Y9M2Z8/ref\\u003dsr_1_5?ie\\u003dUTF8\\u0026qid\\u003d68957459\\u0026sr\\u003d8-1\\u0026keywords\\u003dnull\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"B000Y9M2Z8\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/product/1142716082/ref\\u003ds9_ri_gw_g201_i3?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-2\\u0026pf_rd_r\\u003dF73P3Q3Q42E0MA9IR3WJ\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d544034392\\u0026pf_rd_i\\u003d971120\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1142716082\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/product/1436693578/ref\\u003ds9_simh_gw_p107_d0_i7?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-1\\u0026pf_rd_r\\u003d99O6Y6T9A8GS7R5720B0\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d496554063\\u0026pf_rd_i\\u003d824861\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1436693578\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Stepladder-Fiberglass-ft-375-Capacity/dp/B002AZI49M/ref\\u003dsr_1_1/183-1355335-6997219?ie\\u003dUTF8\\u0026keywords\\u003d0094703196900\\u0026qid\\u003d1434659776\\u0026sr\\u003d8-1\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"B002AZI49M\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/3790815721/ref\\u003dsr_1_1?ie\\u003dUTF8\\u0026qid\\u003d47125543\\u0026sr\\u003d8-1\\u0026keywords\\u003dnull\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"3790815721\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/product/1780830041/ref\\u003ds9_simh_gw_p79_d17_i6?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-1\\u0026pf_rd_r\\u003dN36493T45F7KM5DST4XD\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d681355952\\u0026pf_rd_i\\u003d407137\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1780830041\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/product/1492201278/ref\\u003dsr_1_10?ie\\u003dUTF8\\u0026qid\\u003d60008119\\u0026sr\\u003d8-1\\u0026keywords\\u003dnull\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1492201278\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Moral-Meaning-Bible-1/dp/0976933020/179-3617094-7389914?ie\\u003dUTF8\\u0026redirect\\u003dtrue\\u0026ref_\\u003ds9_simh_gw_p63_d3_i4\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"0976933020\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Artist-Garden-Walled-Great-Glemham/dp/0956186998\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"0956186998\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Power-Techniques-Improve-Memory-Procrastination/dp/1419399381/ref\\u003dsr_1_1/187-0087211-5658230?ie\\u003dUTF8\\u0026keywords\\u003d1419399381\\u0026qid\\u003d1434661016\\u0026sr\\u003d8-1\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1419399381\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/0321982991/ref\\u003dcm_cr_pr_product_top\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"0321982991\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/product/1937303098/ref\\u003ds9_simh_gw_p74_d2_i6?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-2\\u0026pf_rd_r\\u003dPE5RJS9RG897M86VBG55\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d287461703\\u0026pf_rd_i\\u003d730469\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1937303098\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/gp/product/1140905902/ref\\u003dsr_1_3?ie\\u003dUTF8\\u0026qid\\u003d12249211\\u0026sr\\u003d8-1\\u0026keywords\\u003dnull\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1140905902\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/1452668159/ref\\u003dcm_cr_pr_product_top\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1452668159\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/product/0500282234/ref\\u003ds9_simh_gw_p79_d17_i2?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-1\\u0026pf_rd_r\\u003dIR0529FYG68I106DR7SL\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d292698592\\u0026pf_rd_i\\u003d436709\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"0500282234\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Overland-Beyond-Theresa-Hewat/dp/0903909138/ref\\u003dsr_1_1/190-6495656-9434655?ie\\u003dUTF8\\u0026keywords\\u003d0903909138\\u0026qid\\u003d1434660544\\u0026sr\\u003d8-1\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"0903909138\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Vanity-Fair-Coverage-Underwire-75346/dp/B00Q7DN5VU/ref\\u003dsr_1_1/166-8687063-8587826?ie\\u003dUTF8\\u0026keywords\\u003d0083623777443\\u0026qid\\u003d1473951935\\u0026sr\\u003d8-1\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"metaCategId\\\":11450,\\\"leafCategId\\\":63853,\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"B00Q7DN5VU\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Kurt-Adler-Dessert-Recipe-Ornament/dp/B018JQXHPG/ref\\u003dsr_1_2/162-8279833-5061559?ie\\u003dUTF8\\u0026keywords\\u003d0086131332074\\u0026qid\\u003d1473951951\\u0026sr\\u003d8-2\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"metaCategId\\\":1,\\\"leafCategId\\\":148,\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"B018JQXHPG\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Beyond-Geometry-Experiments-Form-1940s-1970s/dp/0262240475\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"0262240475\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/gp/product/162459042X/ref\\u003ds9_ri_gw_g201_i10?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-3\\u0026pf_rd_r\\u003d32MV0S8PFB49KX9B508J\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d645364713\\u0026pf_rd_i\\u003d157500\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"162459042X\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/gp/product/B002OHDRF2/ref\\u003ds9_ri_gw_g201_i10?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-3\\u0026pf_rd_r\\u003dN2S87N6Z398REUVM1QMV\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d456084397\\u0026pf_rd_i\\u003d260036\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"B002OHDRF2\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/ALGEBRA-THIRD-330-H-LANE-BIRKHOFF/dp/B00KIG9HNW/ref\\u003dsr_1_2/175-1006394-2907034?ie\\u003dUTF8\\u0026keywords\\u003d9780821816462\\u0026qid\\u003d1434674736\\u0026sr\\u003d8-2\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"B00KIG9HNW\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/product/0743247612/ref\\u003ds9_simh_gw_p79_d17_i5?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-2\\u0026pf_rd_r\\u003d9QI92D9ZA2PGEU1S67Y8\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d491952717\\u0026pf_rd_i\\u003d133029\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"0743247612\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Building-Procurement-Surveying-Alan-Turner/dp/0333522869/ref\\u003dsr_1_1/180-3512075-9002103?ie\\u003dUTF8\\u0026keywords\\u003d9780333522868\\u0026qid\\u003d1434926616\\u0026sr\\u003d8-1\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"0333522869\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Persecution-Prejudice-Integrated-humanities-David/dp/0582025095/ref\\u003dsr_1_1/192-2160439-7754248?ie\\u003dUTF8\\u0026keywords\\u003d9780582025097\\u0026qid\\u003d1434520194\\u0026sr\\u003d8-1\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"0582025095\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/product/B006GHAPYC/ref\\u003dsr_1_7?ie\\u003dUTF8\\u0026qid\\u003d46028791\\u0026sr\\u003d8-1\\u0026keywords\\u003dnull\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"B006GHAPYC\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Walt-Disney-His-Life-Pictures/dp/0786831162/ref\\u003dsr_1_1/182-6322434-7758236?ie\\u003dUTF8\\u0026keywords\\u003d9780786831166\\u0026qid\\u003d1434675102\\u0026sr\\u003d8-1\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"0786831162\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}"]} \ No newline at end of file diff --git a/measure/src/test/resources/output.msg b/measure/src/test/resources/output.msg new file mode 100644 index 000000000..4ec9288b4 --- /dev/null +++ b/measure/src/test/resources/output.msg @@ -0,0 +1 @@ +{"groups":[{"_name":"DefaultGroup","groupType":"FULLSITE","storeType":"CATALOG","attrsList":[{"name":"CENTERCOL","dataType":"STRING","values":["\u003cdiv id\u003d\"centerCol\" class\u003d\"centerColumn\"\u003e \u003cdiv id\u003d\"booksTitle\" class\u003d\"feature\" data-feature-name\u003d\"booksTitle\"\u003e \u003cdiv class\u003d\"a-section a-spacing-none\"\u003e \u003ch1 id\u003d\"title\" class\u003d\"a-size-large a-spacing-none\"\u003e \u003cspan id\u003d\"productTitle\" class\u003d\"a-size-large\"\u003eBefore the First Day\u003c/span\u003e \u003cspan class\u003d\"a-size-medium a-color-secondary a-text-normal\"\u003ePaperback\u003c/span\u003e \u003c!-- use pre formatted date that complies with legal requirement from media matrix --\u003e \u003cspan class\u003d\"a-size-medium a-color-secondary a-text-normal\"\u003e– July 22, 2013\u003c/span\u003e \u003c/h1\u003e \u003c/div\u003e \u003cdiv id\u003d\"byline\" class\u003d\"a-section a-spacing-micro bylineHidden feature\"\u003e by \u003cspan class\u003d\"author notFaded\" data-width\u003d\"\"\u003e \u003ca class\u003d\"a-link-normal\" href\u003d\"/s/ref\u003ddp_byline_sr_book_1?ie\u003dUTF8\u0026amp;text\u003dStephen+M.+Hale\u0026amp;search-alias\u003dbooks\u0026amp;field-author\u003dStephen+M.+Hale\u0026amp;sort\u003drelevancerank\"\u003eStephen M. Hale\u003c/a\u003e \u003cspan class\u003d\"contribution\" spacing\u003d\"none\"\u003e \u003cspan class\u003d\"a-color-secondary\"\u003e(Author)\u003c/span\u003e \u003c/span\u003e \u003c/span\u003e \u003c/div\u003e \u003c/div\u003e \u003cdiv id\u003d\"averageCustomerReviews_feature_div\" class\u003d\"feature\" data-feature-name\u003d\"averageCustomerReviews\"\u003e \u003cstyle type\u003d\"text/css\"\u003e /* * Fix for UDP-1061. Average customer reviews has a small extra line on hover * https://omni-grok.amazon.com/xref/src/appgroup/websiteTemplates/retail/SoftlinesDetailPageAssets/udp-intl-lock/src/legacy.css?indexName\u003dWebsiteTemplates#40 */ .noUnderline a:hover { text-decoration: none; }\u003c/style\u003e \u003cdiv id\u003d\"averageCustomerReviews\" class\u003d\"a-spacing-none\" data-asin\u003d\"1628391340\" data-ref\u003d\"dpx_acr_pop_\"\u003e \u003cspan class\u003d\"a-declarative\" data-action\u003d\"acrStarsLink-click-metrics\" data-acrstarslink-click-metrics\u003d\"{}\"\u003e \u003cspan id\u003d\"acrPopover\" class\u003d\"reviewCountTextLinkedHistogram noUnderline\" title\u003d\"5.0 out of 5 stars\"\u003e \u003cspan class\u003d\"a-declarative\" data-action\u003d\"a-popover\" data-a-popover\u003d\"{\u0026quot;max-width\u0026quot;:\u0026quot;700\u0026quot;,\u0026quot;closeButton\u0026quot;:\u0026quot;false\u0026quot;,\u0026quot;position\u0026quot;:\u0026quot;triggerBottom\u0026quot;,\u0026quot;url\u0026quot;:\u0026quot;/gp/customer-reviews/widgets/average-customer-review/popover/ref\u003ddpx_acr_pop_?contextId\u003ddpx\u0026amp;asin\u003d1628391340\u0026quot;}\"\u003e \u003ca href\u003d\"javascript:void(0)\" class\u003d\"a-popover-trigger a-declarative\"\u003e \u003ci class\u003d\"a-icon a-icon-star a-star-5\"\u003e\u003cspan class\u003d\"a-icon-alt\"\u003e5.0 out of 5 stars\u003c/span\u003e\u003c/i\u003e \u003ci class\u003d\"a-icon a-icon-popover\"\u003e\u003c/i\u003e\u003c/a\u003e \u003c/span\u003e \u003cspan class\u003d\"a-letter-space\"\u003e\u003c/span\u003e \u003c/span\u003e \u003c/span\u003e \u003cspan class\u003d\"a-letter-space\"\u003e\u003c/span\u003e \u003cspan class\u003d\"a-declarative\" data-action\u003d\"acrLink-click-metrics\" data-acrlink-click-metrics\u003d\"{}\"\u003e \u003ca id\u003d\"acrCustomerReviewLink\" class\u003d\"a-link-normal\" href\u003d\"#customerReviews\"\u003e \u003cspan id\u003d\"acrCustomerReviewText\" class\u003d\"a-size-base\"\u003e1 customer review\u003c/span\u003e \u003c/a\u003e \u003c/span\u003e \u003cscript type\u003d\"text/javascript\"\u003e P.when(\u0027A\u0027, \u0027ready\u0027).execute(function(A) { A.declarative(\u0027acrLink-click-metrics\u0027, \u0027click\u0027, { \"allowLinkDefault\" : true }, function(event){ if(window.ue) { ue.count(\"acrLinkClickCount\", (ue.count(\"acrLinkClickCount\") || 0) + 1); } }); }); \u003c/script\u003e \u003cscript type\u003d\"text/javascript\"\u003e P.when(\u0027A\u0027, \u0027cf\u0027).execute(function(A) { A.declarative(\u0027acrStarsLink-click-metrics\u0027, \u0027click\u0027, { \"allowLinkDefault\" : true }, function(event){ if(window.ue) { ue.count(\"acrStarsLinkWithPopoverClickCount\", (ue.count(\"acrStarsLinkWithPopoverClickCount\") || 0) + 1); } }); }); \u003c/script\u003e \u003c/div\u003e \u003c/div\u003e \u003cdiv id\u003d\"seriesTitle_feature_div\" class\u003d\"feature\" data-feature-name\u003d\"seriesTitle\"\u003e \u003c/div\u003e \u003cdiv id\u003d\"zeitgeistBadge_feature_div\" class\u003d\"feature\" data-feature-name\u003d\"zeitgeistBadge\"\u003e \u003c/div\u003e \u003cdiv id\u003d\"socialFabric_feature_div\" class\u003d\"feature\" data-feature-name\u003d\"socialFabric\"\u003e \u003c/div\u003e \u003chr /\u003e \u003cdiv id\u003d\"adoptedData\" class\u003d\"feature\" data-feature-name\u003d\"adoptedData\"\u003e \u003cstyle\u003e #adoptedDataContainer .campusLogo { height:20px; width:20px; margin-right:1px; } #adoptedDataContainer .campusName { line-height:20px; } #adoptedDataContainer.paddingMicro { padding:4px !important; } #adoptedDataContainer .paddingLeftMicro { padding-left:4px !important; } #adoptedDataContainer .paddingLeftSmall { padding-left:10px !important; } #adoptedDataContainer .paddingRightSmall { padding-right:10px !important; } #adoptedDataContainer .courseListExpanderContainer a:focus { outline:none; } #adoptedDataContainer .courseColumn { width:50%; float:left; } #adoptedDataContainer .textRow { display:inline-block; width:100%; }\u003c/style\u003e \u003c/div\u003e \u003cdiv id\u003d\"MediaMatrix\" class\u003d\"feature\" data-feature-name\u003d\"MediaMatrix\"\u003e \u003cdiv id\u003d\"formats\" class\u003d\"a-section a-spacing-large responsive\"\u003e \u003ca class\u003d\"a-link-expander a-spacing-top-micro a-spacing-small a-size-small\"\u003e \u003ci id\u003d\"formatsIcon\" class\u003d\"a-icon a-icon-expand\"\u003e\u003c/i\u003e \u003cspan id\u003d\"showMoreFormatsPrompt\" class\u003d\"tmmShowPrompt\"\u003eSee all 3 formats and editions\u003c/span\u003e \u003cspan id\u003d\"hideMoreFormatsPrompt\" class\u003d\"tmmHidePrompt\"\u003eHide other formats and editions\u003c/span\u003e \u003c/a\u003e \u003cdiv id\u003d\"twister\" class\u003d\"a-section a-spacing-base a-spacing-top-small a-size-mini\"\u003e \u003c!-- This file is just a replica of table-aiv present in mediaMatrixtemplate package. This has just been picked up and moved and there has not been addition/modification/removal of any logic here . --\u003e \u003ctable class\u003d\"a-normal a-spacing-none title\"\u003e \u003ctbody\u003e \u003ctr\u003e \u003ctd class\u003d\"dp-title-col\"\u003e \u003c/td\u003e \u003ctd class\u003d\"a-color-secondary a-text-right dp-price-col\"\u003e \u003cdiv class\u003d\"a-fixed-right-grid\"\u003e \u003cdiv class\u003d\"a-fixed-right-grid-inner\" style\u003d\"padding-right:50px\"\u003e \u003cdiv class\u003d\"a-fixed-right-grid-col a-col-left\" style\u003d\"padding-right:2.5%;*width:97.1%;float:left;\"\u003e Price \u003c/div\u003e \u003cdiv class\u003d\"a-fixed-right-grid-col a-col-right\" style\u003d\"width:50px;margin-right:-50px;float:left;\"\u003e\u003c/div\u003e \u003c/div\u003e \u003c/div\u003e \u003c/td\u003e \u003ctd class\u003d\"a-color-secondary a-text-right dp-new-col\"\u003e New from \u003c/td\u003e \u003ctd class\u003d\"a-color-secondary a-text-right dp-used-col\"\u003e Used from \u003c/td\u003e \u003c/tr\u003e \u003c/tbody\u003e \u003c/table\u003e \u003cdiv class\u003d\"hr unselected-row\"\u003e\u003c/div\u003e \u003cdiv class\u003d\"top-level unselected-row\"\u003e \u003cspan class\u003d\"a-declarative\" data-action\u003d\"tmm-see-more-editions-click\" data-tmm-see-more-editions-click\u003d\"{\u0026quot;metabindingPlaceHolder\u0026quot;:1,\u0026quot;metabindingPlaceHolderState\u0026quot;:\u0026quot;0\u0026quot;,\u0026quot;metabindingUrl\u0026quot;:\u0026quot;/Before-First-Day-Stephen-Hale-ebook/dp/B00E4WYJT8/ref\u003dtmm_kin_title_0?_encoding\u003dUTF8\u0026amp;amp;qid\u003d\u0026amp;amp;sr\u003d\u0026quot;}\" id\u003d\"declarative_1\"\u003e \u003ctable class\u003d\"a-normal a-spacing-none\"\u003e \u003ctbody\u003e \u003ctr\u003e \u003ctd class\u003d\"dp-title-col\"\u003e \u003c!-- Do not show expander when DVD_redesign weblab is in treatment --\u003e \u003ca class\u003d\"empty-expander title-expander\" style\u003d\"vertical-align:middle;\"\u003e \u003cspan class\u003d\"metabinding-expander\"\u003e\u003c/span\u003e \u003c/a\u003e \u003ca href\u003d\"/Before-First-Day-Stephen-Hale-ebook/dp/B00E4WYJT8/ref\u003dtmm_kin_title_0?_encoding\u003dUTF8\u0026amp;qid\u003d\u0026amp;sr\u003d\" class\u003d\"title-text\"\u003e \u003cspan class\u003d\"a-size-small a-color-base\"\u003eKindle\u003c/span\u003e \u003cspan id\u003d\"tmmSpinnerDiv_1\" style\u003d\"display: none\" class\u003d\"tmmAjaxLoading\"\u003e\u003c/span\u003e \u003c/a\u003e \u003cspan id\u003d\"tmmErrorDiv_1\" class\u003d\"tmmErrorClass displayNone\"\u003e \u003cdiv class\u003d\"a-box a-alert-inline a-alert-inline-error\"\u003e \u003cdiv class\u003d\"a-box-inner a-alert-container\"\u003e \u003ci class\u003d\"a-icon a-icon-alert\"\u003e\u003c/i\u003e \u003cdiv class\u003d\"a-alert-content\"\u003e \u003cspan\u003e\u0026quot;Please retry\u0026quot;\u003c/span\u003e \u003c/div\u003e \u003c/div\u003e \u003c/div\u003e \u003c/span\u003e \u003c/td\u003e \u003ctd class\u003d\"a-text-right dp-price-col\"\u003e \u003cdiv class\u003d\"a-fixed-right-grid\"\u003e \u003cdiv class\u003d\"a-fixed-right-grid-inner\" style\u003d\"padding-right:50px\"\u003e \u003cdiv class\u003d\"a-fixed-right-grid-col a-col-left\" style\u003d\"padding-right:2.5%;*width:97.1%;float:left;\"\u003e \u003ca class\u003d\"a-link-normal\" href\u003d\"/Before-First-Day-Stephen-Hale-ebook/dp/B00E4WYJT8/ref\u003dtmm_kin_title_0?_encoding\u003dUTF8\u0026amp;qid\u003d\u0026amp;sr\u003d\"\u003e \u003cspan class\u003d\"a-size-small a-color-price\"\u003e$7.99\u003c/span\u003e \u003c/a\u003e \u003c/div\u003e \u003cdiv class\u003d\"a-fixed-right-grid-col a-col-right\" style\u003d\"width:50px;margin-right:-50px;float:left;\"\u003e \u003c/div\u003e \u003c/div\u003e \u003c/div\u003e \u003c/td\u003e \u003ctd class\u003d\"a-color-tertiary a-text-right dp-new-col\"\u003e — \u003c/td\u003e \u003ctd class\u003d\"a-color-tertiary a-text-right dp-used-col\"\u003e — \u003c/td\u003e \u003c/tr\u003e \u003c/tbody\u003e \u003c/table\u003e \u003c/span\u003e \u003c/div\u003e \u003cdiv id\u003d\"metabinding_row_top_1\" class\u003d\"second-level\"\u003e\u003c/div\u003e \u003cdiv id\u003d\"metabinding_row_bottom_1\" class\u003d\"second-level\"\u003e\u003c/div\u003e \u003cdiv class\u003d\"hr selected-row\"\u003e\u003c/div\u003e \u003cdiv class\u003d\"top-level selected-row\"\u003e \u003cspan class\u003d\"a-declarative\" data-action\u003d\"tmm-see-more-editions-click\" data-tmm-see-more-editions-click\u003d\"{\u0026quot;metabindingPlaceHolder\u0026quot;:2,\u0026quot;metabindingPlaceHolderState\u0026quot;:\u0026quot;0\u0026quot;,\u0026quot;metabindingUrl\u0026quot;:\u0026quot;#\u0026quot;}\" id\u003d\"declarative_2\"\u003e \u003ctable class\u003d\"a-normal a-spacing-none\"\u003e \u003ctbody\u003e \u003ctr\u003e \u003ctd class\u003d\"dp-title-col\"\u003e \u003c!-- Do not show expander when DVD_redesign weblab is in treatment --\u003e \u003ca class\u003d\"a-link-expander title-expander\" style\u003d\"vertical-align:middle; padding-left:16px\"\u003e \u003cspan class\u003d\"a-link-expander metabinding-expander\"\u003e \u003ci id\u003d\"editionsIcon_2\" class\u003d\"a-icon a-icon-expand tmm-row-icon\"\u003e\u003c/i\u003e \u003c/span\u003e \u003c/a\u003e \u003cspan class\u003d\"title-text\"\u003e \u003cspan class\u003d\"a-size-small a-color-base\"\u003ePaperback\u003c/span\u003e \u003cspan id\u003d\"tmmSpinnerDiv_2\" style\u003d\"display: none\" class\u003d\"tmmAjaxLoading\"\u003e\u003c/span\u003e \u003c/span\u003e \u003cspan id\u003d\"tmmErrorDiv_2\" class\u003d\"tmmErrorClass displayNone\"\u003e \u003cdiv class\u003d\"a-box a-alert-inline a-alert-inline-error\"\u003e \u003cdiv class\u003d\"a-box-inner a-alert-container\"\u003e \u003ci class\u003d\"a-icon a-icon-alert\"\u003e\u003c/i\u003e \u003cdiv class\u003d\"a-alert-content\"\u003e \u003cspan\u003e\u0026quot;Please retry\u0026quot;\u003c/span\u003e \u003c/div\u003e \u003c/div\u003e \u003c/div\u003e \u003c/span\u003e \u003c/td\u003e \u003ctd class\u003d\"a-text-right dp-price-col\"\u003e \u003cdiv class\u003d\"a-fixed-right-grid\"\u003e \u003cdiv class\u003d\"a-fixed-right-grid-inner\" style\u003d\"padding-right:50px\"\u003e \u003cdiv class\u003d\"a-fixed-right-grid-col a-col-left\" style\u003d\"padding-right:2.5%;*width:97.1%;float:left;\"\u003e \u003cspan class\u003d\"a-size-small a-color-price\"\u003e$15.49\u003c/span\u003e \u003c/div\u003e \u003cdiv class\u003d\"a-fixed-right-grid-col a-col-right\" style\u003d\"width:50px;margin-right:-50px;float:left;\"\u003e \u003c/div\u003e \u003c/div\u003e \u003c/div\u003e \u003c/td\u003e \u003ctd class\u003d\"a-text-right dp-new-col\"\u003e \u003ca class\u003d\"a-link-normal\" href\u003d\"/gp/offer-listing/1628391340/ref\u003dtmm_pap_new_olp_sr?ie\u003dUTF8\u0026amp;condition\u003dnew\u0026amp;qid\u003d\u0026amp;sr\u003d\"\u003e \u003cspan\u003e$9.44\u003c/span\u003e \u003c/a\u003e \u003c/td\u003e \u003ctd class\u003d\"a-text-right dp-used-col\"\u003e \u003ca class\u003d\"a-link-normal\" href\u003d\"/gp/offer-listing/1628391340/ref\u003dtmm_pap_used_olp_sr?ie\u003dUTF8\u0026amp;condition\u003dused\u0026amp;qid\u003d\u0026amp;sr\u003d\"\u003e \u003cspan\u003e$12.17\u003c/span\u003e \u003c/a\u003e \u003c/td\u003e \u003c/tr\u003e \u003c/tbody\u003e \u003c/table\u003e \u003c/span\u003e \u003c/div\u003e \u003cdiv id\u003d\"metabinding_row_top_2\" class\u003d\"second-level\"\u003e\u003c/div\u003e \u003cdiv id\u003d\"metabinding_row_bottom_2\" class\u003d\"second-level\"\u003e\u003c/div\u003e \u003cscript type\u003d\"a-state\" data-a-state\u003d\"{\u0026quot;key\u0026quot;:\u0026quot;mediamatrix-state\u0026quot;}\"\u003e{\"url_2\":\"/gp/media-matrix/fetch-DPX-expansion-data.html/ref\u003dtmm_pap_metabinding_expansion_sr?ie\u003dUTF8\u0026amp;tagActionCode\u003d\u0026amp;bindingCount\u003d2\u0026amp;websiteDisplayGroup\u003dbook_display_on_website\u0026amp;cor\u003dUS\u0026amp;storeID\u003d\u0026amp;qid\u003d\u0026amp;productTypeDefinition\u003dABIS_BOOK\u0026amp;viewID\u003dglance\u0026amp;landingAsin\u003d1628391340\u0026amp;customerID\u003d\u0026amp;tasParentAsin\u003dB00E5D53IM\u0026amp;asin\u003d1628391340\u0026amp;metabinding\u003dpaperback_meta_binding\u0026amp;metabindingIndex\u003d2\u0026amp;isPrime\u003d0\u0026amp;nodeID\u003d\u0026amp;sr\u003d\",\"isAjaxInProgress_1\":\"0\",\"url_1\":\"/gp/media-matrix/fetch-DPX-expansion-data.html/ref\u003dtmm_kin_metabinding_expansion_0?ie\u003dUTF8\u0026amp;tagActionCode\u003d\u0026amp;bindingCount\u003d1\u0026amp;websiteDisplayGroup\u003dbook_display_on_website\u0026amp;cor\u003dUS\u0026amp;storeID\u003d\u0026amp;qid\u003d\u0026amp;productTypeDefinition\u003dABIS_BOOK\u0026amp;viewID\u003dglance\u0026amp;landingAsin\u003d1628391340\u0026amp;customerID\u003d\u0026amp;tasParentAsin\u003dB00E5D53IM\u0026amp;asin\u003dB00E4WYJT8\u0026amp;metabinding\u003dkindle_meta_binding\u0026amp;metabindingIndex\u003d1\u0026amp;isPrime\u003d0\u0026amp;nodeID\u003d\u0026amp;sr\u003d\",\"isAjaxInProgress_2\":\"0\",\"isAjaxComplete_1\":\"0\",\"isAjaxComplete_2\":\"0\"}\u003c/script\u003e \u003c/div\u003e \u003cdiv id\u003d\"tmmSwatches\" class\u003d\"a-row nonJSFormats\"\u003e \u003cul class\u003d\"a-nostyle a-button-list a-horizontal\"\u003e \u003cli class\u003d\"swatchElement unselected\"\u003e\u003cspan class\u003d\"a-list-item\"\u003e \u003cspan class\u003d\"a-button a-spacing-mini a-button-toggle format\"\u003e\u003cspan class\u003d\"a-button-inner\"\u003e\u003ca href\u003d\"/Before-First-Day-Stephen-Hale-ebook/dp/B00E4WYJT8/ref\u003dtmm_kin_swatch_0?_encoding\u003dUTF8\u0026amp;qid\u003d\u0026amp;sr\u003d\" class\u003d\"a-button-text\" role\u003d\"button\"\u003e \u003cspan\u003eKindle\u003c/span\u003e \u003cbr /\u003e \u003cspan class\u003d\"a-color-secondary\"\u003e \u003cspan\u003e $7.99 \u003c/span\u003e \u003c/span\u003e \u003c/a\u003e\u003c/span\u003e\u003c/span\u003e \u003cspan class\u003d\"tmm-olp-links\"\u003e \u003c/span\u003e \u003cspan class\u003d\"a-size-mini a-color-secondary tmm-olp-links\"\u003e \u003ca id\u003d\"kcpAppsPopOver\" class\u003d\"a-size-mini a-link-normal kcpAppsPopOver\" href\u003d\"javascript:void(0);\"\u003e \u003cspan class\u003d\"kcpAppBaseBox_\"\u003e \u003cspan class\u003d\"kcpAppsPopOver\"\u003e Read with Our \u003cspan class\u003d\"a-color-price a-text-bold\"\u003e Free App \u003c/span\u003e \u003c/span\u003e \u003c/span\u003e \u003c/a\u003e \u003c/span\u003e \u003cspan class\u003d\"tmm-olp-links\"\u003e \u003c/span\u003e \u003c/span\u003e\u003c/li\u003e \u003cli class\u003d\"swatchElement selected\"\u003e\u003cspan class\u003d\"a-list-item\"\u003e \u003cspan class\u003d\"a-button a-button-selected a-spacing-mini a-button-toggle format\"\u003e\u003cspan class\u003d\"a-button-inner\"\u003e\u003ca href\u003d\"javascript:void(0)\" class\u003d\"a-button-text\" role\u003d\"button\"\u003e \u003cspan\u003ePaperback\u003c/span\u003e \u003cbr /\u003e \u003cspan class\u003d\"a-color-base\"\u003e \u003cspan class\u003d\"a-color-price\"\u003e $15.49 \u003c/span\u003e \u003c/span\u003e \u003c/a\u003e\u003c/span\u003e\u003c/span\u003e \u003cspan class\u003d\"tmm-olp-links\"\u003e \u003c/span\u003e \u003cspan class\u003d\"tmm-olp-links\"\u003e \u003cspan class\u003d\"olp-used olp-link\"\u003e \u003ca class\u003d\"a-size-mini a-link-normal\" href\u003d\"/gp/offer-listing/1628391340/ref\u003dtmm_pap_used_olp_sr?ie\u003dUTF8\u0026amp;condition\u003dused\u0026amp;qid\u003d\u0026amp;sr\u003d\"\u003e 2 Used \u003cspan class\u003d\"olp-from\"\u003efrom\u003c/span\u003e $12.17 \u003c/a\u003e \u003c/span\u003e \u003cspan class\u003d\"olp-new olp-link\"\u003e \u003ca class\u003d\"a-size-mini a-link-normal\" href\u003d\"/gp/offer-listing/1628391340/ref\u003dtmm_pap_new_olp_sr?ie\u003dUTF8\u0026amp;condition\u003dnew\u0026amp;qid\u003d\u0026amp;sr\u003d\"\u003e 13 New \u003cspan class\u003d\"olp-from\"\u003efrom\u003c/span\u003e $9.44 \u003c/a\u003e \u003c/span\u003e \u003c/span\u003e \u003c/span\u003e\u003c/li\u003e \u003c/ul\u003e \u003c/div\u003e \u003c/div\u003e \u003c/div\u003e \u003cdiv id\u003d\"applicablePromotionList_feature_div\" class\u003d\"feature\" data-feature-name\u003d\"applicablePromotionList\"\u003e \u003c/div\u003e \u003cdiv id\u003d\"holidayDeliveryMessage_feature_div\" class\u003d\"feature\" data-feature-name\u003d\"holidayDeliveryMessage\"\u003e \u003c/div\u003e \u003cdiv id\u003d\"campusInfo\" class\u003d\"feature\" data-feature-name\u003d\"campusInfo\"\u003e \u003c/div\u003e \u003cdiv id\u003d\"extraProductInfoFeatureGroup\" class\u003d\"feature\" data-feature-name\u003d\"extraProductInfoFeatureGroup\"\u003e \u003cdiv id\u003d\"newerVersion_feature_div\" class\u003d\"feature\" data-feature-name\u003d\"newerVersion\"\u003e \u003c/div\u003e \u003cdiv id\u003d\"productAlert_feature_div\" class\u003d\"feature\" data-feature-name\u003d\"productAlert\"\u003e \u003c/div\u003e \u003c/div\u003e \u003cdiv id\u003d\"dynamicIframe_feature_div\" class\u003d\"feature\" data-feature-name\u003d\"dynamicIframe\"\u003e \u003cscript id\u003d\"dynamic_iframe_CSS\" type\u003d\"text/undefined\"\u003ehtml { font-size: 100%;}a:focus { outline: thin dotted #333333; outline-offset: -2px;}a:hover, a:active { outline: 0 none;}body { background-color: #FFFFFF; margin:0; /* from AUI */ color: #333333; font-family: Arial,sans-serif; font-size: 13px; line-height: 19px;}h1, h2, h3, h4, h5, h6 { margin: 0; padding: 0;}h1, h2, h3, h4 { font-family: Arial,sans-serif; text-rendering: optimizelegibility; padding-bottom: 4px;}h1:last-child, h2:last-child, h3:last-child, h4:last-child { padding-bottom: 0;}h1 { font-size: 28px; font-weight: normal; line-height: 1.2;}h2 { font-size: 21px; font-weight: normal; line-height: 1.3;}h3 { font-size: 17px; font-weight: normal; line-height: 1.24;}p { margin: 0 0 14px; padding: 0;}p:last-child { margin-bottom: 0;}p + p { margin-top: -4px;}b, strong { font-weight: bold;}i, em { font-style: italic;}blockquote { margin: 13px;}small { font-size: 12px;}img { border: 0 none; max-width: 100%; vertical-align: top;}td { float: none; margin-right: 0;}td, th { padding: 3px; vertical-align: top;}td:first-child, th:first-child { padding-left: 0;}td:last-child, th:last-child { padding-right: 0;}th { text-align: left;}tr:last-child td, tr:last-child th { padding-bottom: 0;}tr:first-child td, tr:first-child th { padding-top: 0;}a, a:visited, a:active, a:link { color: #007EB9; text-decoration: none;}a:hover { color: #E47911; cursor: pointer; text-decoration: underline;}p a { text-decoration: underline;}ul { /*color: #AAAAAA; Overriding this style as the color is out of sync with the rest of book description*/ list-style-type: disc;}ol { color: #888888;}ul, ol { margin: 0 0 18px 22px; padding: 0;}ul li, ol li { margin: 0; word-wrap: break-word;}ul:last-child, ol:last-child { margin-bottom: 0 !important;}#iframeContent {overflow: hidden;}h2.productDescriptionHeader { margin-bottom: 0em;}.emptyClear { clear:left; height:0px; font-size:0px;}div.productDescriptionWrapper { margin: 0 0 1em 0;}h3.productDescriptionSource { font-weight:normal; color:#333333; font-size:1.23em; margin: .75em 0 .375em 0; clear:left;}.seeAll { margin-top: 1.25em; margin-left: -15px;}#technicalProductFeatures ul { list-style-type: disc; margin: 1.12em 0; margin-left: 20px;}#iframeContent ul { list-style-type: disc; margin-left: 20px;}ul li { margin: 0 0 0 20px;}ul li ul { list-style-type: disc; margin-left: 20px;}ul li ul li { margin: 0 0 0 20px;}.aplus h4, .aplus h5 { margin: 0 0 .75em 0; font-size: 1em;}.aplus h4 { color: #CC6600;}.aplus p { margin: 0 0 1em 0;}.aplus .break { clear:both; height:0px; font-size:0px;}.aplus .spacer { margin-bottom: 13px;}.aplus img { border:none;}.aplus .leftImage, .aplus .rightImage, .aplus .centerImage { margin-bottom: 1em; margin-top: 0; text-align:center; vertical-align:top;}.aplus .leftImage { margin-right: 15px; float:left; clear:left;}.aplus .rightImage { margin-left: 15px; float:right; clear:right;}.aplus .imageCaption { clear:both; padding: .5em .5em 0 .5em; font-size: .846em; display: block;}.aplus table.data { border-collapse: collapse; margin-bottom: 1.25em;}.aplus table.data th { font-weight: bold; background: #F7F7F7; border-style:solid; border-color: #CCCCCC; border-width:0 0 1px 1px;}.aplus table.data td { border-left: 1px solid #CCC; border-bottom: 1px dotted #CCC}.aplus table.data th, .aplus table.data td{ padding:3px 10px; text-align:left}.aplus table.data tfoot { font-style: italic;}.aplus table.data caption { background: #eee; font-size: .8125em;}.aplus table.data tr td:first-child, .aplus table.data tr th:first-child { border-left-width:0px;}.aplus ul { margin:0 0 1em 0;}.aplus .center { text-align: center;}.aplus .right { text-align: right;}.aplus .sixth-col,.aplus .fourth-col,.aplus .third-col,.aplus .half-col,.aplus .two-third-col,.aplus .three-fourth-col,.aplus .one-col { float:left; margin-right: 1.6760%; overflow: hidden;}.aplus .last { margin-right:0px;}.aplus .sixth-col { width: 15.080%;}.aplus .fourth-col { width: 23.4637%;}.aplus .third-col { width: 31.8436%;}.aplus .half-col { width: 48.6034%;}.aplus .two-third-col { width: 65.3631%;}.aplus .three-fourth-col { width: 73.7430%;}.aplus .one-col { width: 98.8827%; margin-right:0;}.aplus .last { margin-right:0;}.aplus { width: 100%; min-width: 895px;}\u003c/script\u003e \u003cscript type\u003d\"text/javascript\"\u003eP.register(\"DynamicIframe\", function(){function DynamicIframe(options) { var nTries \u003d 0, MAX_TRIES \u003d 20, iframeId \u003d options.iframeId, encodedIframeContent \u003d options.encodedIframeContent, iframeWrapperId \u003d options.iframeWrapperId, initialResizeCallback \u003d options.initialResizeCallback, iframeCSSId \u003d \"dynamic_iframe_CSS\"; iframeOverriddenCSSId \u003d options.overriddenCSSId; this.createIframe \u003d function() { var iframe \u003d document.createElement(\u0027iframe\u0027); iframe.id \u003d iframeId; iframe.className \u003d \"ap_never_hide\"; iframe.width \u003d \"100%\"; iframe.scrolling \u003d \"no\"; iframe.frameBorder \u003d \"0\"; onloadFn \u003d this.iframeload; if ( iframe.addEventListener ) iframe.addEventListener(\"load\", onloadFn, false); else if ( iframe.attachEvent ) iframe.attachEvent(\"onload\", onloadFn); else iframe.onload \u003d onloadFn; var wrapper \u003d document.getElementById(iframeWrapperId); wrapper.innerHTML \u003d \u0027\u0027; iframe \u003d wrapper.appendChild(iframe); var overriddenCSS \u003d \"\"; if (document.getElementById(iframeOverriddenCSSId)) { \toverriddenCSS \u003d document.getElementById(iframeOverriddenCSSId).innerHTML; } var doc \u003d ((iframe.contentWindow \u0026\u0026 iframe.contentWindow.document) || iframe.contentDocument); if (doc \u0026\u0026 doc.open) { doc.open(); doc.writeln(\u0027\u003chtml\u003e\u003chead\u003e\u003cbase target\u003d\"_top\" /\u003e\u003c/\u0027 + \u0027head\u003e\u003cbody\u003e\u003cstyle class\u003d\"text/css\"\u003e\u0027 + document.getElementById(iframeCSSId).innerHTML + overriddenCSS + \u0027\u003c/style\u003e\u003cdiv id\u003d\"iframeContent\"\u003e\u0027 + \u0027\u0027 + decodeURIComponent(encodedIframeContent) + \u0027\u003c/\u0027+\u0027div\u003e\u0027+\u0027\u003c/\u0027+\u0027body\u003e\u003c/html\u003e\u0027); doc.close(); } } this.iframeload \u003d function () { var iframe \u003d document.getElementById(iframeId); iframe.style.display \u003d \u0027\u0027; setTimeout(function () { setIframeHeight(initialResizeCallback); }, 20); } function getDocHeight(doc) { var contentDiv \u003d doc.getElementById(\"iframeContent\"); var docHeight \u003d 0; if(contentDiv){ docHeight \u003d Math.max( contentDiv.scrollHeight, contentDiv.offsetHeight, contentDiv.clientHeight ); } return docHeight; } function setIframeHeight(resizeCallback) { var iframeDoc, iframe \u003d document.getElementById(iframeId); iframeDoc \u003d ((iframe.contentWindow \u0026\u0026 iframe.contentWindow.document) || iframe.contentDocument); if (iframeDoc) { var h \u003d getDocHeight(iframeDoc); if (h \u0026\u0026 h !\u003d 0) { iframe.style.height \u003d parseInt(h) + \u0027px\u0027; if(typeof resizeCallback \u003d\u003d \"function\") { resizeCallback(iframeId);\t } } else if (nTries \u003c MAX_TRIES) { nTries++; setTimeout(function () { setIframeHeight(resizeCallback); }, 50); } } } this.resizeIframe \u003d function(resizeCallback) { nTries \u003d 0;\tsetIframeHeight(resizeCallback); }}return DynamicIframe;});\u003c/script\u003e \u003c/div\u003e \u003cdiv id\u003d\"bookDescription_feature_div\" class\u003d\"feature\" data-feature-name\u003d\"bookDescription\"\u003e \u003cscript id\u003d\"bookDesc_override_CSS\" type\u003d\"text/undefined\"\u003ebody { font-size: 14px; line-height: 1.6em;}.aplus {\tmin-width: inherit;}\u003c/script\u003e \u003cnoscript\u003e \u003cdiv\u003e This is book one of a fantasy series for youths through adults. It is about Narn, a young man in a village of people that is still recent after the Creation. The Maker has Given him the special ability to sense people\u0027s feelings and has Called him to help people when their feelings would turn them from the Maker\u0027s Teachings. Unbeknownst to him, Narn is being pursued by the Blackness, a force from the unMaker trying to undo all that the Maker has Made and that killed Narn\u0027s Father, who also shared Narn\u0027s special sensitivity. Narn discovers that he must choose between the Lessons his Family learned from the Maker and those of the Hunters, seeded from the unMaker. After his best friend, a Hunter, is killed while on a Hunt, Narn\u0027s own emotions blind him, and he ignores his Lessons to avenge his friend\u0027s death. He then finds that he must choose between life and death, not only for the once-revered beast that killed his friend, but also for himself-because his heart has Writings upon it that conflict with what he has seen take place before him. Which is more real? To which will he yield? This book sets the stage for the following series about Narn, which gradually deepens in the question of \u0026quot;choosing whom you will follow,\u0026quot; and gives the reader insight into the Writings on his/her own heart so he/she may be better prepared to make that choice. I have a master\u0027s degree in counseling psychology with special emphasis in personal and family challenges. I began to see that the underlying causes of most people\u0027s problems stemmed from a departure from Standards in their lives, consciously or unconsciously. This realization influenced me very strongly and I found myself writing about it in story form. \u003c/div\u003e \u003cem\u003e\u003c/em\u003e \u003c/noscript\u003e \u003cdiv id\u003d\"outer_postBodyPS\" style\u003d\"overflow: hidden; z-index: 1; height: 0px; display: block;\"\u003e \u003cdiv id\u003d\"postBodyPS\" style\u003d\"overflow: hidden;\"\u003e \u003cdiv id\u003d\"bookDesc_iframe_wrapper\" class\u003d\"maxReadableWidth\"\u003e\u003c/div\u003e \u003c/div\u003e \u003c/div\u003e \u003cdiv id\u003d\"psPlaceHolder\" style\u003d\"height: 20px; display: none;\"\u003e \u003cdiv style\u003d\"z-index: 3;\"\u003e \u003ca class\u003d\"a-link-expander h2-expander\"\u003e \u003ci id\u003d\"bdExpanderIcon\" class\u003d\"a-icon a-icon-expand\"\u003e\u003c/i\u003e \u003cspan id\u003d\"bdSeeAllPrompt\"\u003eRead more\u003c/span\u003e \u003cspan id\u003d\"bdSeeLessPrompt\" style\u003d\"display: none;\"\u003eRead less\u003c/span\u003e \u003c/a\u003e \u003c/div\u003e \u003c/div\u003e \u003cscript type\u003d\"text/javascript\"\u003e P.when(\u0027DynamicIframe\u0027).execute(function(DynamicIframe){\tvar BookDescriptionIframe \u003d null,\t bookDescEncodedData \u003d \"This%20is%20book%20one%20of%20a%20fantasy%20series%20for%20youths%20through%20adults.%20It%20is%20about%20Narn%2C%20a%20young%20man%20in%20a%20village%20of%20people%20that%20is%20still%20recent%20after%20the%20Creation.%20The%20Maker%20has%20Given%20him%20the%20special%20ability%20to%20sense%20people%27s%20feelings%20and%20has%20Called%20him%20to%20help%20people%20when%20their%20feelings%20would%20turn%20them%20from%20the%20Maker%27s%20Teachings.%20Unbeknownst%20to%20him%2C%20Narn%20is%20being%20pursued%20by%20the%20Blackness%2C%20a%20force%20from%20the%20unMaker%20trying%20to%20undo%20all%20that%20the%20Maker%20has%20Made%20and%20that%20killed%20Narn%27s%20Father%2C%20who%20also%20shared%20Narn%27s%20special%20sensitivity.%20Narn%20discovers%20that%20he%20must%20choose%20between%20the%20Lessons%20his%20Family%20learned%20from%20the%20Maker%20and%20those%20of%20the%20Hunters%2C%20seeded%20from%20the%20unMaker.%20After%20his%20best%20friend%2C%20a%20Hunter%2C%20is%20killed%20while%20on%20a%20Hunt%2C%20Narn%27s%20own%20emotions%20blind%20him%2C%20and%20he%20ignores%20his%20Lessons%20to%20avenge%20his%20friend%27s%20death.%20He%20then%20finds%20that%20he%20must%20choose%20between%20life%20and%20death%2C%20not%20only%20for%20the%20once-revered%20beast%20that%20killed%20his%20friend%2C%20but%20also%20for%20himself-because%20his%20heart%20has%20Writings%20upon%20it%20that%20conflict%20with%20what%20he%20has%20seen%20take%20place%20before%20him.%20Which%20is%20more%20real%3F%20To%20which%20will%20he%20yield%3F%20This%20book%20sets%20the%20stage%20for%20the%20following%20series%20about%20Narn%2C%20which%20gradually%20deepens%20in%20the%20question%20of%20%22choosing%20whom%20you%20will%20follow%2C%22%20and%20gives%20the%20reader%20insight%20into%20the%20Writings%20on%20his%2Fher%20own%20heart%20so%20he%2Fshe%20may%20be%20better%20prepared%20to%20make%20that%20choice.%20I%20have%20a%20master%27s%20degree%20in%20counseling%20psychology%20with%20special%20emphasis%20in%20personal%20and%20family%20challenges.%20I%20began%20to%20see%20that%20the%20underlying%20causes%20of%20most%20people%27s%20problems%20stemmed%20from%20a%20departure%20from%20Standards%20in%20their%20lives%2C%20consciously%20or%20unconsciously.%20This%20realization%20influenced%20me%20very%20strongly%20and%20I%20found%20myself%20writing%20about%20it%20in%20story%20form.\",\t bookDescriptionAvailableHeight,\t minBookDescriptionInitialHeight \u003d 112,\t options \u003d {},\t iframeId \u003d \"bookDesc_iframe\"; function resizeCallback() { P.guardFatal(\"bookDescription\", function() { // Get the line-height of the iframe var iframe \u003d document.getElementById(iframeId); var iframeDocument \u003d iframe.contentDocument; if (false \u0026\u0026 iframeDocument \u0026\u0026 iframeDocument.defaultView) { // Set the height to the number of lines specified var numLines \u003d parseInt(0, 10); // Get the line-height of the iframe var iframeContent \u003d iframeDocument.getElementById(\"iframeContent\"); // Compute the line height var lineHeight \u003d iframeDocument.defaultView.getComputedStyle(iframeContent, null).getPropertyValue(\"line-height\"); // Parse the line height lineHeight \u003d parseFloat(lineHeight); bookDescriptionAvailableHeight \u003d Math.round(lineHeight * numLines); } else { var bdOffsetTop \u003d document.getElementById(\"bookDescription_feature_div\").offsetTop; var imageBlockOffsetTop \u003d document.getElementById(\"booksImageBlock_feature_div\").offsetTop; var imageBlockHeight \u003d document.getElementById(\"booksImageBlock_feature_div\").offsetHeight; bookDescriptionAvailableHeight \u003d imageBlockOffsetTop + imageBlockHeight -bdOffsetTop - 30; if(bookDescriptionAvailableHeight \u003c minBookDescriptionInitialHeight) { bookDescriptionAvailableHeight \u003d minBookDescriptionInitialHeight; } } \t\t var psTotalHeight \u003d document.getElementById(\"postBodyPS\").offsetHeight; if(psTotalHeight \u003e bookDescriptionAvailableHeight + 30){ if(document.getElementById(\"bdSeeLessPrompt\").style.display \u003d\u003d \"none\"){ document.getElementById(\"outer_postBodyPS\").style.height \u003d bookDescriptionAvailableHeight + \u0027px\u0027; document.getElementById(\"psPlaceHolder\").style.display \u003d\"block\"; document.getElementById(\"bdSeeAllPrompt\").style.display \u003d\"block\"; } else{ document.getElementById(\"outer_postBodyPS\").style.height \u003d psTotalHeight + \u0027px\u0027; } } else{ document.getElementById(\"outer_postBodyPS\").style.height \u003d psTotalHeight + \u0027px\u0027; document.getElementById(\"psPlaceHolder\").style.display \u003d\"none\"; document.getElementById(\"bdSeeAllPrompt\").style.display \u003d\"block\"; document.getElementById(\"bdSeeLessPrompt\").style.display \u003d\"none\"; document.getElementById(\"bdExpanderIcon\").className \u003d document.getElementById(\"bdExpanderIcon\").className.replace(\"rotate\",\"\"); } })(); } options.iframeId \u003d iframeId; options.iframeWrapperId \u003d \"bookDesc_iframe_wrapper\";\toptions.overriddenCSSId \u003d \"bookDesc_override_CSS\";\toptions.encodedIframeContent \u003d bookDescEncodedData;\toptions.initialResizeCallback \u003d resizeCallback;\t\tBookDescriptionIframe \u003d new DynamicIframe(options);\tP.guardFatal(\"bookDescription\", function() {\t BookDescriptionIframe.createIframe();\t}) ();\t \tif ((typeof BookDescriptionIframe !\u003d \u0027undefined\u0027) \u0026\u0026 (BookDescriptionIframe instanceof DynamicIframe)) {\t P.when(\u0027jQuery\u0027).execute(function($) {\t $(window).resize(function() {\t P.guardFatal(\"bookDescription\", function() {\t BookDescriptionIframe.resizeIframe(resizeCallback);\t }) ();\t });\t $(window).bind(\u0027imageResize\u0027, function() {\t P.guardFatal(\"bookDescription\", function() {\t BookDescriptionIframe.resizeIframe(resizeCallback);\t }) ();\t });\t });\t}});\u003c/script\u003e \u003c/div\u003e \u003cdiv id\u003d\"edpIngress_feature_div\" class\u003d\"feature\" data-feature-name\u003d\"edpIngress\"\u003e \u003c/div\u003e \u003cdiv id\u003d\"heroQuickPromoBooksAtf_feature_div\" class\u003d\"feature\" data-feature-name\u003d\"heroQuickPromoBooksAtf\"\u003e \u003cdiv id\u003d\"hero-quick-promo\" class\u003d\"a-row a-spacing-medium\"\u003e \u003chr class\u003d\"a-spacing-medium a-divider-normal\" /\u003e \u003c!--wlhqp--\u003e \u003cdiv class\u003d\"qpImage\"\u003e \u003ca href\u003d\"/gp/redirect.html/ref\u003damb_link_475358582_1/154-5469300-3928621?ie\u003dUTF8\u0026amp;location\u003dhttp%3A%2F%2Fwww.omnivoracious.com%2F\u0026amp;source\u003dstandards\u0026amp;token\u003dDEF1D0758E667C928314D8D6645CBCC4A8DBCD89\u0026amp;pf_rd_m\u003dATVPDKIKX0DER\u0026amp;pf_rd_s\u003dhero-quick-promo-books-atf\u0026amp;pf_rd_r\u003dW7WQP2BDJ090798P990F\u0026amp;pf_rd_t\u003d201\u0026amp;pf_rd_p\u003d2546723202\u0026amp;pf_rd_i\u003d1628391340\"\u003e\u003cimg src\u003d\"https://images-na.ssl-images-amazon.com/images/G/01/img15/books/other/17002_books_ABR-hqp_75x75._SL75_CB330842061_.jpg\" width\u003d\"75\" align\u003d\"left\" alt\u003d\"The Amazon Book Review\" height\u003d\"75\" border\u003d\"0\" /\u003e\u003c/a\u003e \u003c/div\u003e \u003cspan class\u003d\"qpHeadline\"\u003eThe Amazon Book Review\u003c/span\u003e \u003cbr /\u003e Author interviews, book reviews, editors picks, and more. \u003ca href\u003d\"/gp/redirect.html/ref\u003damb_link_475358582_3/154-5469300-3928621?ie\u003dUTF8\u0026amp;location\u003dhttp%3A%2F%2Fwww.omnivoracious.com%2F\u0026amp;source\u003dstandards\u0026amp;token\u003dDEF1D0758E667C928314D8D6645CBCC4A8DBCD89\u0026amp;pf_rd_m\u003dATVPDKIKX0DER\u0026amp;pf_rd_s\u003dhero-quick-promo-books-atf\u0026amp;pf_rd_r\u003dW7WQP2BDJ090798P990F\u0026amp;pf_rd_t\u003d201\u0026amp;pf_rd_p\u003d2546723202\u0026amp;pf_rd_i\u003d1628391340\"\u003eRead it now\u003c/a\u003e \u003c/div\u003e \u003cdiv style\u003d\"clear:left; margin-bottom:5px\"\u003e\u003c/div\u003e \u003c/div\u003e \u003cdiv id\u003d\"promotionalBundle_feature_div\" class\u003d\"feature\" data-feature-name\u003d\"promotionalBundle\"\u003e \u003c/div\u003e \u003cdiv id\u003d\"andonCord_feature_div\" class\u003d\"feature\" data-feature-name\u003d\"andonCord\"\u003e \u003c/div\u003e \u003cdiv id\u003d\"OosRecommendation_feature_div\" class\u003d\"feature\" data-feature-name\u003d\"OosRecommendation\"\u003e \u003c/div\u003e \u003c/div\u003e"],"defaultValue":"","isMetaAttribute":false,"extractedValuesCount":1},{"name":"IMAGE","dataType":"STRING","values":[""],"defaultValue":"","isMetaAttribute":false,"extractedValuesCount":0},{"name":"SELLER_INFO","dataType":"STRING","values":[""],"defaultValue":"","isMetaAttribute":false,"extractedValuesCount":0},{"name":"PRODUCT_DETAILS","dataType":"STRING","values":["\u003cdiv id\u003d\"detail-bullets\"\u003e \u003ca name\u003d\"productDetails\" id\u003d\"productDetails\"\u003e\u003c/a\u003e \u003chr noshade\u003d\"noshade\" size\u003d\"1\" class\u003d\"bucketDivider\" /\u003e \u003ctable cellpadding\u003d\"0\" cellspacing\u003d\"0\" border\u003d\"0\" id\u003d\"productDetailsTable\"\u003e \u003ctbody\u003e \u003ctr\u003e \u003ctd class\u003d\"bucket\"\u003e \u003ch2\u003eProduct Details\u003c/h2\u003e \u003cdiv class\u003d\"content\"\u003e \u003cul\u003e \u003cli\u003e\u003cb\u003ePaperback:\u003c/b\u003e 200 pages\u003c/li\u003e \u003cli\u003e\u003cb\u003ePublisher:\u003c/b\u003e Xulon Press (July 22, 2013)\u003c/li\u003e \u003cli\u003e\u003cb\u003eLanguage:\u003c/b\u003e English\u003c/li\u003e \u003cli\u003e\u003cb\u003eISBN-10:\u003c/b\u003e 1628391340\u003c/li\u003e \u003cli\u003e\u003cb\u003eISBN-13:\u003c/b\u003e 978-1628391343\u003c/li\u003e \u003cli\u003e\u003cb\u003e Product Dimensions: \u003c/b\u003e 6.1 x 0.4 x 9.2 inches \u003c/li\u003e \u003cli\u003e\u003cb\u003eShipping Weight:\u003c/b\u003e 10.2 ounces (\u003ca href\u003d\"/gp/help/seller/shipping.html/ref\u003ddp_pd_shipping/154-5469300-3928621?ie\u003dUTF8\u0026amp;asin\u003d1628391340\u0026amp;seller\u003dATVPDKIKX0DER\"\u003eView shipping rates and policies\u003c/a\u003e)\u003c/li\u003e \u003cli\u003e\u003cb\u003eAverage Customer Review:\u003c/b\u003e \u003cscript type\u003d\"text/javascript\"\u003efunction acrPopoverHover(e, h) { if(h) window.acrAsinHover \u003d e; else if(window.acrAsinHover \u003d\u003d e) window.acrAsinHover \u003d null;}function popoverReady(jQuery) { if((typeof jQuery \u003d\u003d\u003d \"undefined\") || (typeof jQuery.fn \u003d\u003d\u003d \"undefined\")) { return; } else if(jQuery.fn.acrPopover) { return; } var popoverConfig \u003d { showOnHover:true, showCloseButton: true, width: null, location:\u0027bottom\u0027, locationAlign:\u0027left\u0027, locationOffset:[-20,0], paddingLeft: 15, paddingBottom: 5, paddingRight: 15, group: \u0027reviewsPopover\u0027, clone:false, hoverHideDelay:300 }; jQuery.fn.acrPopover \u003d function() { return this.each(function() { var $this \u003d jQuery(this); if(!$this.data(\u0027init\u0027) \u0026\u0026 typeof $this.amazonPopoverTrigger \u003d\u003d\u003d \u0027function\u0027) { $this.data(\u0027init\u0027, 1); var getargs \u003d $this.attr(\u0027getargs\u0027); var ajaxURL \u003d \u0027/gp/customer-reviews/common/du/displayHistoPopAjax.html?\u0027 + \u0027\u0026ASIN\u003d\u0027 + $this.attr(\u0027name\u0027) + \u0027\u0026link\u003d1\u0027 + \u0027\u0026seeall\u003d1\u0027 + \u0027\u0026ref\u003d\u0027 + $this.attr(\u0027ref\u0027) + ((typeof getargs !\u003d \u0027undefined\u0027) ? (\u0027\u0026getargs\u003d\u0027 + getargs) : \u0027\u0027); var myConfig \u003d jQuery.extend(true, { destination: ajaxURL }, popoverConfig); $this.amazonPopoverTrigger(myConfig); var w \u003d window.acrAsinHover; if(w \u0026\u0026 jQuery(w).parents(\u0027.asinReviewsSummary\u0027).get(0) \u003d\u003d this) { $this.trigger(\u0027mouseover.amzPopover\u0027); window.acrAsinHover \u003d null; } } }); }; window.reviewHistPopoverConfig \u003d popoverConfig; var jqInit \u003d window.jQueryInitHistoPopovers \u003d function(asin) { if (typeof jQuery(\u0027.acr-popover[name\u003d\u0027 + asin + \u0027]\u0027).acrPopover \u003d\u003d\u003d \u0027function\u0027) { jQuery(\u0027.acr-popover[name\u003d\u0027 + asin + \u0027]\u0027).acrPopover(); } }; window.doInit_average_customer_reviews \u003d jqInit; window.onAjaxUpdate_average_customer_reviews \u003d jqInit; window.onCacheUpdate_average_customer_reviews \u003d jqInit; window.onCacheUpdateReselect_average_customer_reviews \u003d jqInit; if (typeof P !\u003d\u003d \u0027undefined\u0027) { P.when(\u0027amznJQ.criticalFeature\u0027).execute(\"amznJQ.criticalFeatureComplete\", function() { setTimeout(function() { P.register(\u0027acrPopover\u0027, function () { return window.arcPopover; }); }, 10); }); } else if (typeof amznJQ !\u003d\u003d \u0027undefined\u0027) { amznJQ.onCompletion(\u0027amznJQ.criticalFeature\u0027, function() { setTimeout(function() { amznJQ.declareAvailable(\u0027acrPopover\u0027); }, 10); }); }};function acrPopoverReady(jQuery) { jQuery(\u0027.acr-popover,#searchTemplate .asinReviewsSummary\u0027).each(function() { if (typeof jQuery(this).acrPopover \u003d\u003d\u003d \u0027function\u0027) { jQuery(this).acrPopover(); } });};if (typeof P !\u003d\u003d \u0027undefined\u0027) { var jq \u003d null; if (typeof $ !\u003d\u003d \u0027undefined\u0027) { jq \u003d $; } else if (typeof jQuery !\u003d\u003d \u0027undefined\u0027) { jq \u003d jQuery; } if (jq) { P.when(\u0027popover\u0027, \u0027ready\u0027).execute(popoverReady(jq)); P.when(\u0027acrPopover\u0027, \u0027ready\u0027).execute(acrPopoverReady(jq)); }} else if ((typeof amznJQ !\u003d\u003d \u0027undefined\u0027) \u0026\u0026 (typeof jQuery !\u003d\u003d \u0027undefined\u0027)) { amznJQ.onReady(\u0027popover\u0027, popoverReady, jQuery); amznJQ.onReady(\u0027acrPopover\u0027, acrPopoverReady, jQuery);};\u003c/script\u003e \u003cspan class\u003d\"crAvgStars\" style\u003d\"white-space:no-wrap;\"\u003e\u003cspan class\u003d\"asinReviewsSummary acr-popover\" name\u003d\"1628391340\" ref\u003d\"dp_db_cm_cr_acr_pop_\"\u003e \u003ca style\u003d\"cursor:pointer;text-decoration:none\" href\u003d\"https://www.amazon.com/Before-First-Day-Stephen-Hale/product-reviews/1628391340/ref\u003ddp_db_cm_cr_acr_img/154-5469300-3928621?ie\u003dUTF8\u0026amp;showViewpoints\u003d1\" name\u003d\"reviewHistoPop_1628391340__star__\" onmouseover\u003d\"return acrPopoverHover(this,1);\" onmouseout\u003d\"return acrPopoverHover(this,0);\"\u003e\u003cspan class\u003d\"swSprite s_star_5_0 \" title\u003d\"5.0 out of 5 stars\"\u003e\u003cspan\u003e5.0 out of 5 stars\u003c/span\u003e\u003c/span\u003e\u0026nbsp;\u003c/a\u003e\u0026nbsp;\u003cspan class\u003d\"histogramButton\" style\u003d\"margin-left:-3px\"\u003e\u003ca style\u003d\"cursor:pointer\" href\u003d\"https://www.amazon.com/Before-First-Day-Stephen-Hale/product-reviews/1628391340/ref\u003ddp_db_cm_cr_acr_img/154-5469300-3928621?ie\u003dUTF8\u0026amp;showViewpoints\u003d1\" name\u003d\"reviewHistoPop_1628391340__button__\" onmouseover\u003d\"return acrPopoverHover(this,1);\" onmouseout\u003d\"return acrPopoverHover(this,0);\"\u003e\u003cspan class\u003d\"swSprite s_chevron \"\u003e\u003cspan\u003eSee all reviews\u003c/span\u003e\u003c/span\u003e\u0026nbsp;\u003c/a\u003e\u003c/span\u003e\u003c/span\u003e(\u003ca href\u003d\"https://www.amazon.com/Before-First-Day-Stephen-Hale/product-reviews/1628391340/ref\u003ddp_db_cm_cr_acr_txt/154-5469300-3928621?ie\u003dUTF8\u0026amp;showViewpoints\u003d1\"\u003e1 customer review\u003c/a\u003e)\u003c/span\u003e\u003c/li\u003e \u003cli id\u003d\"SalesRank\"\u003e \u003cb\u003eAmazon Best Sellers Rank:\u003c/b\u003e #9,398,863 in Books (\u003ca href\u003d\"https://www.amazon.com/best-sellers-books-Amazon/zgbs/books/ref\u003dpd_dp_ts_b_1/154-5469300-3928621\"\u003eSee Top 100 in Books\u003c/a\u003e) \u003cstyle type\u003d\"text/css\"\u003e.zg_hrsr { margin: 0; padding: 0; list-style-type: none; }.zg_hrsr_item { margin: 0 0 0 10px; }.zg_hrsr_rank { display: inline-block; width: 80px; text-align: right; }\u003c/style\u003e \u003cul class\u003d\"zg_hrsr\"\u003e \u003cli class\u003d\"zg_hrsr_item\"\u003e \u003cspan class\u003d\"zg_hrsr_rank\"\u003e#6112\u003c/span\u003e \u003cspan class\u003d\"zg_hrsr_ladder\"\u003ein\u0026nbsp;\u003ca href\u003d\"https://www.amazon.com/best-sellers-books-Amazon/zgbs/books/ref\u003dpd_zg_hrsr_b_1_1/154-5469300-3928621\"\u003eBooks\u003c/a\u003e \u0026gt; \u003ca href\u003d\"https://www.amazon.com/gp/bestsellers/books/12290/ref\u003dpd_zg_hrsr_b_1_2/154-5469300-3928621\"\u003eChristian Books \u0026amp; Bibles\u003c/a\u003e \u0026gt; \u003ca href\u003d\"https://www.amazon.com/gp/bestsellers/books/172806/ref\u003dpd_zg_hrsr_b_1_3/154-5469300-3928621\"\u003eLiterature \u0026amp; Fiction\u003c/a\u003e \u0026gt; \u003cb\u003e\u003ca href\u003d\"https://www.amazon.com/gp/bestsellers/books/7259433011/ref\u003dpd_zg_hrsr_b_1_4_last/154-5469300-3928621\"\u003eFantasy\u003c/a\u003e\u003c/b\u003e\u003c/span\u003e \u003c/li\u003e \u003cli class\u003d\"zg_hrsr_item\"\u003e \u003cspan class\u003d\"zg_hrsr_rank\"\u003e#105141\u003c/span\u003e \u003cspan class\u003d\"zg_hrsr_ladder\"\u003ein\u0026nbsp;\u003ca href\u003d\"https://www.amazon.com/best-sellers-books-Amazon/zgbs/books/ref\u003dpd_zg_hrsr_b_2_1/154-5469300-3928621\"\u003eBooks\u003c/a\u003e \u0026gt; \u003ca href\u003d\"https://www.amazon.com/gp/bestsellers/books/17/ref\u003dpd_zg_hrsr_b_2_2/154-5469300-3928621\"\u003eLiterature \u0026amp; Fiction\u003c/a\u003e \u0026gt; \u003ca href\u003d\"https://www.amazon.com/gp/bestsellers/books/10134/ref\u003dpd_zg_hrsr_b_2_3/154-5469300-3928621\"\u003eGenre Fiction\u003c/a\u003e \u0026gt; \u003cb\u003e\u003ca href\u003d\"https://www.amazon.com/gp/bestsellers/books/12489/ref\u003dpd_zg_hrsr_b_2_4_last/154-5469300-3928621\"\u003eReligious \u0026amp; Inspirational\u003c/a\u003e\u003c/b\u003e\u003c/span\u003e \u003c/li\u003e \u003cli class\u003d\"zg_hrsr_item\"\u003e \u003cspan class\u003d\"zg_hrsr_rank\"\u003e#1162374\u003c/span\u003e \u003cspan class\u003d\"zg_hrsr_ladder\"\u003ein\u0026nbsp;\u003ca href\u003d\"https://www.amazon.com/best-sellers-books-Amazon/zgbs/books/ref\u003dpd_zg_hrsr_b_3_1/154-5469300-3928621\"\u003eBooks\u003c/a\u003e \u0026gt; \u003cb\u003e\u003ca href\u003d\"https://www.amazon.com/gp/bestsellers/books/22/ref\u003dpd_zg_hrsr_b_3_2_last/154-5469300-3928621\"\u003eReligion \u0026amp; Spirituality\u003c/a\u003e\u003c/b\u003e\u003c/span\u003e \u003c/li\u003e \u003c/ul\u003e \u003c/li\u003e \u003c/ul\u003e \u003cspan class\u003d\"tiny\"\u003e \u003cul class\u003d\"noteBullets\"\u003e \u003c/ul\u003e \u003c/span\u003e \u003c/div\u003e \u003c/td\u003e \u003c/tr\u003e \u003c/tbody\u003e \u003c/table\u003e \u003c/div\u003e"],"defaultValue":"","isMetaAttribute":false,"extractedValuesCount":1},{"name":"TECH_DETAILS","dataType":"STRING","values":[""],"defaultValue":"","isMetaAttribute":false,"extractedValuesCount":0},{"name":"CAPTCHA","dataType":"STRING","values":[""],"defaultValue":"","isMetaAttribute":false,"extractedValuesCount":0},{"name":"PRODUCT_IDENTIFIER","dataType":"STRING","values":[""],"defaultValue":"","isMetaAttribute":false,"extractedValuesCount":0},{"name":"BUYBOX","dataType":"STRING","values":["\u003cdiv id\u003d\"unifiedBuyBox_feature_div\" class\u003d\"feature\" data-feature-name\u003d\"unifiedBuyBox\"\u003e \u003cdiv id\u003d\"combinedBuyBox\" class\u003d\"a-section a-spacing-medium\"\u003e \u003cform method\u003d\"post\" id\u003d\"addToCart\" action\u003d\"/gp/product/handle-buy-box/ref\u003ddp_start-bbf_1_glance\" class\u003d\"a-content\"\u003e \u003cinput type\u003d\"hidden\" id\u003d\"session-id\" name\u003d\"session-id\" value\u003d\"154-5469300-3928621\" /\u003e \u003cinput type\u003d\"hidden\" id\u003d\"ASIN\" name\u003d\"ASIN\" value\u003d\"1628391340\" /\u003e \u003cinput type\u003d\"hidden\" id\u003d\"offerListingID\" name\u003d\"offerListingID\" value\u003d\"P0YRnR9szBbdW1hK5VhfhfwyIoQrnhuEd%2BPy%2BBgCbCzNct9GAa8y5i0LbWFzbHdQ3KzbWuQMLfVSXokLOyYlLfXo60wCnhwcYD%2BQpFk1lcQBrvHvvzu1mw%3D%3D\" /\u003e \u003cinput type\u003d\"hidden\" id\u003d\"isMerchantExclusive\" name\u003d\"isMerchantExclusive\" value\u003d\"0\" /\u003e \u003cinput type\u003d\"hidden\" id\u003d\"merchantID\" name\u003d\"merchantID\" value\u003d\"ATVPDKIKX0DER\" /\u003e \u003cinput type\u003d\"hidden\" id\u003d\"isAddon\" name\u003d\"isAddon\" value\u003d\"0\" /\u003e \u003cinput type\u003d\"hidden\" id\u003d\"nodeID\" name\u003d\"nodeID\" value\u003d\"283155\" /\u003e \u003cinput type\u003d\"hidden\" id\u003d\"sellingCustomerID\" name\u003d\"sellingCustomerID\" value\u003d\"A2R2RITDJNW1Q6\" /\u003e \u003cinput type\u003d\"hidden\" id\u003d\"qid\" name\u003d\"qid\" value\u003d\"\" /\u003e \u003cinput type\u003d\"hidden\" id\u003d\"sr\" name\u003d\"sr\" value\u003d\"\" /\u003e \u003cinput type\u003d\"hidden\" id\u003d\"storeID\" name\u003d\"storeID\" value\u003d\"books\" /\u003e \u003cinput type\u003d\"hidden\" id\u003d\"tagActionCode\" name\u003d\"tagActionCode\" value\u003d\"283155\" /\u003e \u003cinput type\u003d\"hidden\" id\u003d\"viewID\" name\u003d\"viewID\" value\u003d\"glance\" /\u003e \u003cinput type\u003d\"hidden\" id\u003d\"rsid\" name\u003d\"rsid\" value\u003d\"154-5469300-3928621\" /\u003e \u003cinput type\u003d\"hidden\" id\u003d\"sourceCustomerOrgListID\" name\u003d\"sourceCustomerOrgListID\" value\u003d\"\" /\u003e \u003cinput type\u003d\"hidden\" id\u003d\"sourceCustomerOrgListItemID\" name\u003d\"sourceCustomerOrgListItemID\" value\u003d\"\" /\u003e \u003cinput type\u003d\"hidden\" name\u003d\"wlPopCommand\" value\u003d\"\" /\u003e \u003cdiv id\u003d\"buybox\" class\u003d\"a-row a-spacing-medium\"\u003e \u003cdiv id\u003d\"rbbContainer\" class\u003d\"a-box-group dp-accordion\"\u003e \u003cdiv class\u003d\"a-box rbbSection selected dp-accordion-active\"\u003e \u003cdiv class\u003d\"a-box-inner\"\u003e \u003cdiv class\u003d\"a-section a-spacing-none a-padding-none\"\u003e \u003cdiv id\u003d\"buyNewSection\" class\u003d\"rbbHeader dp-accordion-row\"\u003e \u003ci class\u003d\"a-icon a-icon-radio-active\"\u003e\u003c/i\u003e \u003ca class\u003d\"a-link-normal rbbHeaderLink\" href\u003d\"/gp/product/1628391340?selectObb\u003dnew\"\u003e \u003ch5\u003e \u003cdiv class\u003d\"a-row\"\u003e \u003cdiv class\u003d\"a-column a-span4 a-text-left a-nowrap\"\u003e \u003cspan class\u003d\"a-text-bold\"\u003eBuy New\u003c/span\u003e \u003c/div\u003e \u003cdiv class\u003d\"a-column a-span8 a-text-right a-span-last\"\u003e \u003cdiv class\u003d\"inlineBlock-display\"\u003e \u003cspan class\u003d\"a-letter-space\"\u003e\u003c/span\u003e \u003cspan class\u003d\"a-size-medium a-color-price offer-price a-text-normal\"\u003e$15.49\u003c/span\u003e \u003c/div\u003e \u003c/div\u003e \u003c/div\u003e \u003c/h5\u003e \u003c/a\u003e \u003c/div\u003e \u003cdiv id\u003d\"buyNewInner\" class\u003d\"rbbContent dp-accordion-inner\"\u003e \u003cdiv id\u003d\"buyBoxInner\" class\u003d\"a-section a-spacing-none\"\u003e \u003cdiv class\u003d\"a-row\"\u003e \u003cdiv class\u003d\"a-column a-span5\"\u003e \u003cdiv id\u003d\"selectQuantity\" class\u003d\"a-section a-spacing-none a-padding-none\"\u003e \u003cspan class\u003d\"a-declarative\" data-action\u003d\"quantity-dropdown\" data-quantity-dropdown\u003d\"{}\"\u003e \u003cspan class\u003d\"a-dropdown-container\"\u003e\u003clabel for\u003d\"quantity\" class\u003d\"a-native-dropdown\"\u003eQty:\u003c/label\u003e\u003cselect name\u003d\"quantity\" autocomplete\u003d\"off\" id\u003d\"quantity\" tabindex\u003d\"-1\" class\u003d\"a-native-dropdown\"\u003e \u003coption value\u003d\"1\" selected\u003d\"\"\u003e1 \u003c/option\u003e \u003coption value\u003d\"2\"\u003e2 \u003c/option\u003e \u003coption value\u003d\"3\"\u003e3 \u003c/option\u003e \u003coption value\u003d\"4\"\u003e4 \u003c/option\u003e \u003coption value\u003d\"5\"\u003e5 \u003c/option\u003e \u003coption value\u003d\"6\"\u003e6 \u003c/option\u003e \u003coption value\u003d\"7\"\u003e7 \u003c/option\u003e \u003coption value\u003d\"8\"\u003e8 \u003c/option\u003e \u003coption value\u003d\"9\"\u003e9 \u003c/option\u003e \u003coption value\u003d\"10\"\u003e10 \u003c/option\u003e \u003coption value\u003d\"11\"\u003e11 \u003c/option\u003e \u003coption value\u003d\"12\"\u003e12 \u003c/option\u003e \u003coption value\u003d\"13\"\u003e13 \u003c/option\u003e \u003coption value\u003d\"14\"\u003e14 \u003c/option\u003e \u003coption value\u003d\"15\"\u003e15 \u003c/option\u003e \u003coption value\u003d\"16\"\u003e16 \u003c/option\u003e \u003coption value\u003d\"17\"\u003e17 \u003c/option\u003e \u003coption value\u003d\"18\"\u003e18 \u003c/option\u003e \u003coption value\u003d\"19\"\u003e19 \u003c/option\u003e \u003coption value\u003d\"20\"\u003e20 \u003c/option\u003e \u003coption value\u003d\"21\"\u003e21 \u003c/option\u003e \u003coption value\u003d\"22\"\u003e22 \u003c/option\u003e \u003coption value\u003d\"23\"\u003e23 \u003c/option\u003e \u003coption value\u003d\"24\"\u003e24 \u003c/option\u003e \u003coption value\u003d\"25\"\u003e25 \u003c/option\u003e \u003coption value\u003d\"26\"\u003e26 \u003c/option\u003e \u003coption value\u003d\"27\"\u003e27 \u003c/option\u003e \u003coption value\u003d\"28\"\u003e28 \u003c/option\u003e \u003coption value\u003d\"29\"\u003e29 \u003c/option\u003e \u003coption value\u003d\"30\"\u003e30 \u003c/option\u003e \u003c/select\u003e\u003cspan tabindex\u003d\"-1\" class\u003d\"a-button a-button-dropdown a-button-small\"\u003e\u003cspan class\u003d\"a-button-inner\"\u003e\u003cspan class\u003d\"a-button-text a-declarative\" data-action\u003d\"a-dropdown-button\" role\u003d\"button\" tabindex\u003d\"0\" aria-hidden\u003d\"true\"\u003e\u003cspan class\u003d\"a-dropdown-label\"\u003eQty:\u003c/span\u003e\u003cspan class\u003d\"a-dropdown-prompt\"\u003e1\u003c/span\u003e\u003c/span\u003e\u003ci class\u003d\"a-icon a-icon-dropdown\"\u003e\u003c/i\u003e\u003c/span\u003e\u003c/span\u003e\u003c/span\u003e \u003c/span\u003e \u003c/div\u003e \u003c/div\u003e \u003cdiv class\u003d\"a-column a-span7 a-text-right a-span-last\"\u003e \u003cul class\u003d\"a-nostyle a-vertical\"\u003e \u003c/ul\u003e \u003c/div\u003e \u003c/div\u003e \u003c/div\u003e \u003cdiv class\u003d\"a-section a-spacing-small a-spacing-top-micro\"\u003e \u003cdiv class\u003d\"a-row\"\u003e \u003cspan class\u003d\"a-color-base buyboxShippingLabel\"\u003e \u003c!-- MsgId:cfs_free_shipping_eligible_no_popover_link:web --\u003e \u003ca href\u003d\"/gp/help/customer/display.html/ref\u003dmk_sss_dp_1?ie\u003dUTF8\u0026amp;pop-up\u003d1\u0026amp;nodeId\u003d527692\" target\u003d\"AmazonHelp\" onclick\u003d\"return amz_js_PopWin(this.href,\u0027AmazonHelp\u0027,\u0027width\u003d550,height\u003d550,resizable\u003d1,scrollbars\u003d1,toolbar\u003d0,status\u003d0\u0027);\"\u003eFREE Shipping\u003c/a\u003e on orders with at least $25 of books.\u003c/span\u003e \u003c/div\u003e \u003c/div\u003e \u003cdiv class\u003d\"a-section a-spacing-none\"\u003e \u003c/div\u003e \u003cdiv class\u003d\"a-section a-spacing-small\"\u003e \u003cdiv id\u003d\"availability\" class\u003d\"a-section a-spacing-none\"\u003e \u003cspan class\u003d\"a-size-medium a-color-success\"\u003e May take an extra 1-2 days to ship. \u003c/span\u003e \u003c/div\u003e \u003cdiv class\u003d\"a-section a-spacing-none\"\u003e \u003c/div\u003e \u003cdiv id\u003d\"merchant-info\" class\u003d\"a-section a-spacing-mini\"\u003e Ships from and sold by Amazon.com. \u003cspan class\u003d\"\"\u003e Gift-wrap available. \u003c/span\u003e \u003c/div\u003e \u003c/div\u003e \u003cdiv id\u003d\"bbopAndCartBox\" class\u003d\"a-box\"\u003e \u003cdiv class\u003d\"a-box-inner\"\u003e \u003cstyle type\u003d\"text/css\"\u003e#bbop-sbbop-container { margin-bottom: 0px;}\u003c/style\u003e \u003cdiv id\u003d\"bbop-sbbop-container\" class\u003d\"a-section\"\u003e \u003cdiv id\u003d\"sbbop-popover-header\" class\u003d\"a-section a-hidden\"\u003e \u003cdiv class\u003d\"a-box a-alert-inline a-alert-inline-success\"\u003e \u003cdiv class\u003d\"a-box-inner a-alert-container\"\u003e \u003ci class\u003d\"a-icon a-icon-alert\"\u003e\u003c/i\u003e \u003cdiv class\u003d\"a-color-base\"\u003e Before the First Day has been added to your Cart \u003c/div\u003e \u003c/div\u003e \u003c/div\u003e \u003c/div\u003e \u003c/div\u003e \u003cscript type\u003d\"text/javascript\"\u003eif( window.P ){ P.when(\"A\").execute(function(A) { if (window.sbbopLoaded \u003d\u003d\u003d undefined) { window.sbbopLoaded \u003d false; } function getQueryParameterByName(name) { name \u003d name.replace(/[\\[]/, \"\\\\[\").replace(/[\\]]/, \"\\\\]\"); var regex \u003d new RegExp(\"[\\\\?\u0026]\" + name + \"\u003d([^\u0026#]*)\"), results \u003d regex.exec(location.search); return results \u003d\u003d null ? \"\" : decodeURIComponent(results[1].replace(/\\+/g, \" \")); } var hidden, visibilityChange; if (typeof document.hidden !\u003d\u003d \"undefined\") { hidden \u003d \"hidden\"; visibilityChange \u003d \"visibilitychange\"; } else if (typeof document.mozHidden !\u003d\u003d \"undefined\") { hidden \u003d \"mozHidden\"; visibilityChange \u003d \"mozvisibilitychange\"; } else if (typeof document.msHidden !\u003d\u003d \"undefined\") { hidden \u003d \"msHidden\"; visibilityChange \u003d \"msvisibilitychange\"; } else if (typeof document.webkitHidden !\u003d\u003d \"undefined\") { hidden \u003d \"webkitHidden\"; visibilityChange \u003d \"webkitvisibilitychange\"; } var loadFeatures \u003d function() { var $ \u003d A.$, $container \u003d $(\"#bbop-sbbop-container\"); var data \u003d { \"ASIN\" : \"1628391340\", \"merchantID\" : \"ATVPDKIKX0DER\", \"bbopruleID\" : \"Acquisition_AddToCart_PrimeBasicFreeTrialUpsellEligible\", \"sbbopruleID\" : \"Acquisition_AddToCart_PrimeBasicFreeTrialUpsellEligible\", \"deliveryOptions\" : \"[next,std-n-us,sss-us,second]\", \"preorder\" : \"false\", \"releaseDateDeliveryEligible\" : \"false\" }; var scope \u003d \"bbopAjaxCall\"; if(typeof uet \u003d\u003d\u003d \u0027function\u0027) { uet(\u0027bb\u0027, scope, {wb: 1}); } $.get(\"/gp/product/du/bbop-ms3-ajax-endpoint.html\", data, function(result) { $container.append(result); if(typeof uet \u003d\u003d\u003d \u0027function\u0027 \u0026\u0026 typeof uex \u003d\u003d\u003d \u0027function\u0027) { uet( \"cf\", scope, {wb: 1} ); uex( \"ld\", scope, {wb: 1} ); } if (getQueryParameterByName(\"checkBBOP\") \u003d\u003d\u003d \"true\") { P.when(\"a-modal\", \"ready\").execute(function(modal) { $(\u0027#bbop-check-box\u0027).click(); }); } loadSBBOP(); }); var loadSBBOP \u003d function(){ P.when(\"a-modal\", \"ready\").execute(function(modal) { if(!window.sbbopLoaded){ var sbbop_modal \u003d createModal(modal); if (sbbop_modal !\u003d\u003d undefined) { $(document.body).delegate(\u0027#add-to-cart-button\u0027,\u0027click.sbbop\u0027, function(event){ $(document.body).undelegate(\u0027#add-to-cart-button\u0027,\u0027click.sbbop\u0027); var $sbbop \u003d $(\"#sbbop-container\"), $sbbopContent \u003d $sbbop.find(\"#sbbop-popover-content\"), $sbbopHeader \u003d $container.find(\"#sbbop-popover-header\"), $sbbopFooter \u003d $sbbop.find(\"#sbbop-popover-footer\"), isAsinizationCase \u003d $sbbop.find(\".is-sbbop-asinization-usecase\").length \u003e 0, $bbopCheckbox \u003d $(\u0027#bbop-check-box\u0027), $atcButton \u003d $(\u0027#add-to-cart-button\u0027), atcName \u003d $atcButton.attr(\"name\"), hasSingleYesButton \u003d ($(\u0027#sbbop-yes-button\u0027).length \u003d\u003d\u003d 1), hasSingleNoButton \u003d ($(\u0027#sbbop-no-button\u0027).length \u003d\u003d\u003d 1), validAtcButton \u003d ($atcButton.length \u003d\u003d\u003d 1 \u0026\u0026 atcName !\u003d \"submit.add-to-cart-prime-buy-box.x\"), validSbbopContent \u003d ($sbbopContent.length \u003d\u003d\u003d 1 \u0026\u0026 $sbbopHeader.length \u003d\u003d\u003d 1 \u0026\u0026 $sbbopFooter.length \u003d\u003d\u003d 1), validSbbopButtons \u003d (hasSingleYesButton \u0026\u0026 hasSingleNoButton), bbopIsNotChecked \u003d !$bbopCheckbox.is(\":checked\"); if (validAtcButton \u0026\u0026 validSbbopContent \u0026\u0026 validSbbopButtons \u0026\u0026 bbopIsNotChecked) { $atcButton.bind(\u0027click.sbbop\u0027, function(){return false;}); $(document.body).delegate(\"#sbbop-yes-button\", \"click\", function(event){ if(isAsinizationCase){ $bbopCheckbox.prop(\"checked\",true); logMetric(\"CSM_prime_surprise_BBOP_yes_button_HO\"); } else { $atcButton.attr(\"name\", \"submit.add-to-cart-prime-buy-box.x\"); logMetric(\"CSM_prime_surprise_BBOP_yes_button\"); } safeATCClick(); }); $(document.body).delegate(\"#sbbop-no-button\", \"click\", function(event){ if(isAsinizationCase) { logMetric(\"CSM_prime_surprise_BBOP_no_button_HO\"); } else { logMetric(\"CSM_prime_surprise_BBOP_no_button\"); } safeATCClick(); }); $(document.body).delegate(\".a-popover\", \"keydown\", function(event){ if(event.keyCode \u003d\u003d \"27\"){ event.preventDefault(); } }); A.on(\"a:popover:ajaxFail:sbbop_modal\", safeATCClick); A.on(\"a:popover:hide:sbbop_modal\", function(){ if(isAsinizationCase) { logMetric(\"CSM_prime_surprise_BBOP_close_button_HO\"); } else { logMetric(\"CSM_prime_surprise_BBOP_close_button\"); } safeATCClick(); }); A.on(\"a:popover:beforeShow:sbbop_modal\", function(){ setEventEmbu(); }); sbbop_modal.show(); if(isAsinizationCase) { logMetric(\"CSM_prime_surprise_BBOP_presentation_HO\"); } else { logMetric(\"CSM_prime_surprise_BBOP_presentation\"); } return false; } }); } window.sbbopLoaded \u003d true; } }); }; var createModal \u003d function(modal) { var $sbbop \u003d $(\"#sbbop-container\"), $sbbopContent \u003d $sbbop.find(\"#sbbop-popover-content\"), $sbbopHeader \u003d $container.find(\"#sbbop-popover-header\"), $sbbopFooter \u003d $sbbop.find(\"#sbbop-popover-footer\"); if($sbbopContent.length \u003d\u003d\u003d 1 \u0026\u0026 $sbbopHeader.length \u003d\u003d\u003d 1 \u0026\u0026 $sbbopFooter.length \u003d\u003d\u003d 1){ sbbopWidth \u003d parseInt(($sbbop.find(\"#sbbop-popover\")).attr(\"popover-width\"), 10); return modal.create($sbbop, { \"inlineContent\": $sbbopContent.html(), \"header\": $sbbopHeader.html(), \"footer\": $sbbopFooter.html(), \"width\": sbbopWidth, \"activate\": \"onclick\", \"name\": \"sbbop_modal\" }); } return undefined; } var logMetric \u003d function(customTag){ if(window.ue \u0026\u0026 ue.tag) { ue.tag(customTag, \"surpriseBBOP\") } }; var postBackMetrics \u003d function(){ if (window.ue){ uex(\"ld\", \"surpriseBBOP\"); } }; var setEventEmbu \u003d function(){ var eventData \u003d { \"eventCode\" : \"106\" }; $.get(\"/gp/prime/utility/record-embu-event.html\", eventData); }; var safeATCClick \u003d function() { postBackMetrics(); $(\u0027#add-to-cart-button\u0027).unbind(\u0027click.sbbop\u0027) .click() .bind(\u0027click.sbbop\u0027,function(){return false}); }; } var handleVisibilityChange \u003d function() { if (!document[hidden]) { document.removeEventListener(visibilityChange, handleVisibilityChange); loadFeatures(); } } if (typeof document.addEventListener \u003d\u003d\u003d \"undefined\" || typeof document[hidden] \u003d\u003d\u003d \"undefined\" || !document[hidden]) { loadFeatures(); } else { document.addEventListener(visibilityChange, handleVisibilityChange); } });}\u003c/script\u003e \u003cdiv class\u003d\"a-button-stack\"\u003e \u003cspan id\u003d\"submit.add-to-cart\" class\u003d\"a-button a-spacing-small a-button-primary a-button-icon\"\u003e\u003cspan class\u003d\"a-button-inner\"\u003e\u003ci class\u003d\"a-icon a-icon-cart\"\u003e\u003c/i\u003e\u003cinput id\u003d\"add-to-cart-button\" name\u003d\"submit.add-to-cart\" title\u003d\"Add to Shopping Cart\" data-hover\u003d\"Select \u0026lt;b\u0026gt;__dims__\u0026lt;/b\u0026gt; from the left\u0026lt;br\u0026gt; to add to Shopping Cart\" class\u003d\"a-button-input\" type\u003d\"submit\" value\u003d\"Add to Cart\" aria-labelledby\u003d\"submit.add-to-cart-announce\" /\u003e\u003cspan id\u003d\"submit.add-to-cart-announce\" class\u003d\"a-button-text\" aria-hidden\u003d\"true\"\u003eAdd to Cart\u003c/span\u003e\u003c/span\u003e\u003c/span\u003e \u003c/div\u003e \u003c/div\u003e \u003c/div\u003e \u003cdiv class\u003d\"a-row a-spacing-none\"\u003e \u003cdiv id\u003d\"oneClickSignIn\" class\u003d\"a-section a-spacing-none\"\u003e \u003cdiv class\u003d\"a-divider a-divider-break a-spacing-micro\"\u003e \u003ch5\u003e\u003ca href\u003d\"/gp/product/utility/edit-one-click-pref.html?ie\u003dUTF8\u0026amp;query\u003dselectObb%3dnew\u0026amp;returnPath\u003d%2fgp%2fproduct%2f1628391340\"\u003e\u003cspan class\u003d\"a-size-mini\"\u003eTurn on 1-Click ordering for this browser\u003c/span\u003e\u003c/a\u003e\u003c/h5\u003e \u003c/div\u003e \u003c/div\u003e \u003c/div\u003e \u003cdiv class\u003d\"a-row\"\u003e \u003cdiv id\u003d\"dpFastTrack_feature_div\" data-feature-name\u003d\"dpFastTrack\" data-template-name\u003d\"dpFastTrack\" class\u003d\"a-section a-spacing-none a-spacing-top-small feature\"\u003e \u003c!-- We only want padding, if the weblab is on--\u003e \u003cdiv id\u003d\"fast-track\" class\u003d\"a-section a-spacing-none\"\u003e \u003cinput type\u003d\"hidden\" id\u003d\"ftSelectAsin\" value\u003d\"1628391340\" /\u003e \u003cinput type\u003d\"hidden\" id\u003d\"ftSelectMerchant\" value\u003d\"ATVPDKIKX0DER\" /\u003e \u003cdiv id\u003d\"fast-track-message\" class\u003d\"a-section a-spacing-none\"\u003e \u003c/div\u003e \u003cscript type\u003d\"text/javascript\"\u003e P.when(\"A\", \"jQuery\").execute(function(A, $) { $(\"#quantity\").live(\"change\", function (event) { if (event.updateFTOnQuantityChange) { return; } event.updateFTOnQuantityChange \u003d 1; var quantity \u003d $(this).val(); var asin \u003d $(\"#ftSelectAsin\").val(); var merchantId \u003d $(\"#ftSelectMerchant\").val(); if (!asin || !merchantId) { return; } var params \u003d []; params.push(\"asin\u003d\" + asin); params.push(\"quantity\u003d\" + quantity); params.push(\"merchantId\u003d\" + merchantId); $.ajax({ type: \"POST\", url: \"/gp/product/features/dp-fast-track/udp-ajax-handler/get-quantity-update-message.html?ie\u003dUTF8\", contentType: \u0027application/x-www-form-urlencoded;charset\u003dutf-8\u0027, data: params.join(\u0027\u0026\u0027), dataType: \"html\", success: function(objResponse) { if (objResponse !\u003d null \u0026\u0026 objResponse !\u003d \"\") { $(\"#fast-track-message\").replaceWith(objResponse); } } }); return; }); });\u003c/script\u003e \u003c/div\u003e \u003c/div\u003e \u003c/div\u003e \u003cdiv id\u003d\"unifiedLocation_feature_div\" data-feature-name\u003d\"unifiedLocation\" data-template-name\u003d\"unifiedLocation\" class\u003d\"a-section feature unifiedLocationMarginBottomClass\"\u003e \u003cscript type\u003d\"text/javascript\"\u003e P.now(\"LOCUX_DETALPAGE_51079_T1\").execute(function(LOCUX_DETALPAGE_51079_T1) { if (!LOCUX_DETALPAGE_51079_T1) { P.declare(\"LOCUX_DETALPAGE_51079_T1\", {}); } }); \u003c/script\u003e \u003cdiv class\u003d\"a-section a-spacing-none\"\u003e \u003chr class\u003d\"a-divider-normal\" /\u003e \u003cdiv class\u003d\"a-section a-spacing-mini\"\u003e \u003cspan class\u003d\"a-text-bold\"\u003e Ship to: \u003c/span\u003e \u003c/div\u003e \u003cdiv class\u003d\"a-section a-spacing-none\"\u003e \u003cspan class\u003d\"a-declarative\" data-action\u003d\"a-popover\" data-a-popover\u003d\"{\u0026quot;closeButton\u0026quot;:\u0026quot;false\u0026quot;,\u0026quot;name\u0026quot;:\u0026quot;LUXAddressSelector\u0026quot;,\u0026quot;activate\u0026quot;:\u0026quot;onclick\u0026quot;}\"\u003e \u003ca href\u003d\"javascript:void(0)\" class\u003d\"a-popover-trigger a-declarative\"\u003e \u003cspan class\u003d\"a-color-base lux-location-label\"\u003e DALLAS, TX 75201 \u003c/span\u003e \u003ci class\u003d\"a-icon a-icon-popover\"\u003e\u003c/i\u003e\u003c/a\u003e \u003c/span\u003e \u003c/div\u003e \u003cdiv class\u003d\"a-popover-preload\" id\u003d\"a-popover-LUXAddressSelector\"\u003e \u003c!-- Section to show to unrecognized customers (instead of address selector) --\u003e \u003cdiv id\u003d\"unifiedLocationTitleNoAddress\" class\u003d\"a-section a-spacing-small a-spacing-top-small a-text-center\"\u003e \u003cspan class\u003d\"a-text-bold\"\u003e To see addresses, please \u003c/span\u003e \u003cbr /\u003e \u003cspan id\u003d\"unifiedLocationSignIn\" class\u003d\"a-button a-spacing-top-base a-button-primary\"\u003e\u003cspan class\u003d\"a-button-inner\"\u003e\u003cinput class\u003d\"a-button-input\" type\u003d\"submit\" aria-labelledby\u003d\"unifiedLocationSignIn-announce\" /\u003e\u003cspan id\u003d\"unifiedLocationSignIn-announce\" class\u003d\"a-button-text\" aria-hidden\u003d\"true\"\u003e Sign in \u003c/span\u003e\u003c/span\u003e\u003c/span\u003e \u003c/div\u003e \u003c!-- Place holder for anchoring the address selector --\u003e \u003cdiv id\u003d\"lux-address-selector-anchor\" class\u003d\"a-section a-spacing-none\"\u003e\u003c/div\u003e \u003cdiv class\u003d\"a-section a-spacing-none lux-address-selector-divider\"\u003e \u003c/div\u003e \u003cdiv id\u003d\"unifiedLocationAddrInputDiv\" class\u003d\"a-section a-spacing-none\"\u003e \u003cdiv class\u003d\"a-divider a-divider-break lux-zip-selector-divider\"\u003e \u003ch5\u003eor\u003c/h5\u003e \u003c/div\u003e \u003clabel class\u003d\"a-text-bold\"\u003e Use this location: \u003c/label\u003e \u003cdiv class\u003d\"a-row a-spacing-top-micro\" role\u003d\"form\"\u003e \u003cdiv class\u003d\"a-column a-span8\" role\u003d\"form\"\u003e \u003cinput type\u003d\"text\" maxlength\u003d\"5\" id\u003d\"unifiedLocationAddrInput\" placeholder\u003d\"Enter US zip\" class\u003d\"a-input-text a-span12\" /\u003e \u003c/div\u003e \u003cdiv class\u003d\"a-column a-span4 a-span-last\" role\u003d\"button\"\u003e \u003cspan id\u003d\"unifiedLocationAddrUpdate\" class\u003d\"a-button a-button-span12\"\u003e\u003cspan class\u003d\"a-button-inner\"\u003e\u003cinput class\u003d\"a-button-input\" type\u003d\"submit\" aria-labelledby\u003d\"unifiedLocationAddrUpdate-announce\" /\u003e\u003cspan id\u003d\"unifiedLocationAddrUpdate-announce\" class\u003d\"a-button-text\" aria-hidden\u003d\"true\"\u003e Update \u003c/span\u003e\u003c/span\u003e\u003c/span\u003e \u003c/div\u003e \u003c/div\u003e \u003cdiv id\u003d\"unifiedLocationInvalidInputAlert\" class\u003d\"a-row a-spacing-top-small aok-hidden\"\u003e \u003cspan class\u003d\"a-color-error\"\u003e Please enter a valid US zip code. \u003c/span\u003e \u003c/div\u003e \u003cdiv id\u003d\"LUXInvalidZipCodeAlert\" class\u003d\"a-row a-spacing-top-small aok-hidden\" role\u003d\"alert\"\u003e \u003cdiv class\u003d\"a-box a-alert-inline a-alert-inline-error a-spacing-none\"\u003e \u003cdiv class\u003d\"a-box-inner a-alert-container\"\u003e \u003ci class\u003d\"a-icon a-icon-alert\"\u003e\u003c/i\u003e \u003cdiv class\u003d\"a-alert-content\"\u003e Please enter a valid US zip code. \u003c/div\u003e \u003c/div\u003e \u003c/div\u003e \u003c/div\u003e \u003cdiv id\u003d\"LUXMilitaryZipCodeAlert\" class\u003d\"a-row a-spacing-top-small aok-hidden\" role\u003d\"alert\"\u003e \u003cdiv class\u003d\"a-box a-alert-inline a-alert-inline-info a-spacing-none\"\u003e \u003cdiv class\u003d\"a-box-inner a-alert-container\"\u003e \u003ci class\u003d\"a-icon a-icon-alert\"\u003e\u003c/i\u003e \u003cdiv class\u003d\"a-alert-content\"\u003e Shipping to a APO/FPO/DPO? Please add the address to your address book. Make sure you include the unit and box numbers (if assigned). \u003c/div\u003e \u003c/div\u003e \u003c/div\u003e \u003c/div\u003e \u003c/div\u003e \u003cdiv class\u003d\"a-divider a-divider-break a-spacing-base a-spacing-top-base lux-country-selector-divider\"\u003e \u003ch5\u003eor\u003c/h5\u003e \u003c/div\u003e \u003cdiv id\u003d\"unifiedLocationCountrySelectionDiv\" class\u003d\"a-row a-spacing-none a-spacing-top-small\"\u003e \u003c/div\u003e \u003c/div\u003e \u003cscript type\u003d\"text/javascript\"\u003e var shouldPopulateCountrySelector \u003d true; P.now(\"LOCUX_DETAILPAGE_68144_T1\").execute(function(LOCUX_DETAILPAGE_68144_T1) { if(LOCUX_DETAILPAGE_68144_T1) { shouldPopulateCountrySelector \u003d true; } }); var fullPageRefresh \u003d false; P.now(\"LOCUX_FULLPAGE_REFRESH\").execute(function(LOCUX_FULLPAGE_REFRESH) { if(LOCUX_FULLPAGE_REFRESH) { fullPageRefresh \u003d true; } }); P.now(\"LUXContext\").execute(function(LUXContext) { if (!LUXContext) { P.declare(\"LUXContext\", { isRecognizedCustomer: false, deviceType: \"web\", isOneClick: false, clientPage: \"DPX\", shouldPopulateCountrySelector: shouldPopulateCountrySelector, fullPageRefresh: fullPageRefresh }); } }); P.when(\"LUXController\").execute(function(LUXController) { LUXController.init({ addressLabel : \"\", addressId : \"\", obfuscatedId : \"\", countryCode : \"US\", zipCode : \"75201\", state : \"TX\", city : \"DALLAS\", district : \"\", locationType : \"IP2LOCATION\" }); }); \u003c/script\u003e \u003c/div\u003e \u003cinput type\u003d\"hidden\" name\u003d\"dropdown-selection\" value\u003d\"add-new\" id\u003d\"unifiedLocationAddress\" data-addnewaddress\u003d\"add-new\" /\u003e \u003cscript type\u003d\"text/javascript\"\u003e P.now(\"LUXDPOnly\").execute(function(DP) { DP || P.declare(\"LUXDPOnly\", {}); }); P.when(\"LUXDPRefreshController\").execute(function(Controller) { Controller.init({ asin : \"1628391340\", merchantId : \"ATVPDKIKX0DER\", deviceType : \"web\", smid : \"\", exMerchId : \"\", wdg: \"book_display_on_website\", oneClick : false, addressId : \"\", obfuscatedId : \"\", city : \"DALLAS\", state : \"TX\", countryCode : \"US\", zipCode : \"75201\", district : \"\", useTwisterRefresh : \"C\", updateAddressForNon1Click : \"T1\", relatedRequestId : \"W7WQP2BDJ090798P990F\" }); }); \u003c/script\u003e \u003c/div\u003e \u003cdiv class\u003d\"a-row\"\u003e \u003cdiv id\u003d\"holidayAvailabilityMessage_feature_div\" data-feature-name\u003d\"holidayAvailabilityMessage\" data-template-name\u003d\"holidayAvailabilityMessage\" class\u003d\"a-section a-spacing-top-small feature\"\u003e \u003c/div\u003e \u003c/div\u003e \u003c/div\u003e \u003c/div\u003e \u003c/div\u003e \u003c/div\u003e \u003cdiv class\u003d\"a-box rbbSection unselected\"\u003e \u003cdiv class\u003d\"a-box-inner\"\u003e \u003cdiv class\u003d\"a-section a-spacing-none a-padding-none\"\u003e \u003cdiv id\u003d\"usedBuySection\" class\u003d\"rbbHeader dp-accordion-row\"\u003e \u003ci class\u003d\"a-icon a-icon-radio-inactive\"\u003e\u003c/i\u003e \u003ca class\u003d\"a-link-normal rbbHeaderLink\" href\u003d\"/gp/product/1628391340?selectObb\u003dused\"\u003e \u003ch5\u003e \u003cdiv class\u003d\"a-row\"\u003e \u003cdiv class\u003d\"a-column a-span4 a-text-left a-nowrap\"\u003e \u003cspan class\u003d\"a-text-bold\"\u003eBuy Used\u003c/span\u003e \u003c/div\u003e \u003cdiv class\u003d\"a-column a-span8 a-text-right a-span-last\"\u003e \u003cdiv class\u003d\"inlineBlock-display\"\u003e \u003cspan class\u003d\"a-letter-space\"\u003e\u003c/span\u003e \u003cspan class\u003d\"a-color-base offer-price a-text-normal\"\u003e$12.94\u003c/span\u003e \u003c/div\u003e \u003c/div\u003e \u003c/div\u003e \u003c/h5\u003e \u003c/a\u003e \u003c/div\u003e \u003cdiv id\u003d\"usedbuyBox\" class\u003d\"a-section rbbContent dp-accordion-inner\"\u003e \u003cinput type\u003d\"hidden\" id\u003d\"usedMerchantID\" name\u003d\"usedMerchantID\" value\u003d\"A3QP1Q1OVJQM8C\" /\u003e \u003cinput type\u003d\"hidden\" id\u003d\"usedOfferListingID\" name\u003d\"usedOfferListingID\" value\u003d\"NTBvnTZwEVBK9r5EhFUVCKqW6Ex8W7A9MKFOezgEz53HNLJ1CYQZ5WjObDRVVYtNWyo7jZdkdj3gRmS7XVPmO9aFJSyenmNJC5DvMY93A4w3j5T5kE%2FqxoqIOlVilcqdkAiQYyDDFjK7HhbKPpL86g%3D%3D\" /\u003e \u003cinput type\u003d\"hidden\" id\u003d\"usedSellingCustomerID\" name\u003d\"usedSellingCustomerID\" value\u003d\"A3QP1Q1OVJQM8C\" /\u003e \u003cdiv class\u003d\"a-section a-spacing-mini\"\u003e \u003cdiv class\u003d\"a-row\"\u003e +\u0026nbsp;$3.99\u0026nbsp;shipping \u003c/div\u003e \u003c/div\u003e \u003cdiv class\u003d\"a-section a-spacing-base\"\u003e \u003cdiv class\u003d\"a-row\"\u003e \u003cstrong\u003e Used: Like New \u003c/strong\u003e \u003cspan class\u003d\"a-size-base\"\u003e \u003cspan class\u003d\"a-color-tertiary\"\u003e | \u003c/span\u003e \u003ca id\u003d\"usedItemConditionInfoLink\" class\u003d\"a-link-normal\" href\u003d\"#\"\u003e Details \u003c/a\u003e \u003c/span\u003e \u003c/div\u003e \u003cdiv class\u003d\"a-row\"\u003e Sold by \u003ca class\u003d\"a-link-normal\" href\u003d\"/gp/help/seller/at-a-glance.html?ie\u003dUTF8\u0026amp;seller\u003dA3QP1Q1OVJQM8C\"\u003ePrepbooks\u003c/a\u003e \u003c/div\u003e \u003c/div\u003e \u003cdiv class\u003d\"a-popover-preload\" id\u003d\"a-popover-usedItemConditionDetailsPopover\"\u003e \u003cdiv class\u003d\"a-section a-spacing-micro\"\u003e \u003cspan class\u003d\"a-size-mini\"\u003e \u003cstrong\u003eCondition:\u003c/strong\u003e Used: Like New \u003c/span\u003e \u003c/div\u003e \u003cdiv class\u003d\"a-section a-spacing-micro\"\u003e \u003cspan class\u003d\"a-size-mini\"\u003e \u003cstrong\u003eComment:\u003c/strong\u003e Multiple Quantities. Fast International Shipping. Tracking Number Provided on all Orders. Quick Order Processing. Ships from Multiple US Locations based on Location and Availability. Satisfaction Guaranteed. \u003c/span\u003e \u003c/div\u003e \u003c/div\u003e \u003cdiv class\u003d\"a-section accessCode-spacing\"\u003e \u003c/div\u003e \u003cdiv class\u003d\"a-button-stack\"\u003e \u003cspan id\u003d\"submit.add-to-cart-ubb\" class\u003d\"a-button a-spacing-small a-button-primary a-button-icon\"\u003e\u003cspan class\u003d\"a-button-inner\"\u003e\u003ci class\u003d\"a-icon a-icon-cart\"\u003e\u003c/i\u003e\u003cinput id\u003d\"add-to-cart-button-ubb\" name\u003d\"submit.add-to-cart-ubb\" title\u003d\"Add to Shopping Cart\" data-hover\u003d\"Select \u0026lt;b\u0026gt;__dims__\u0026lt;/b\u0026gt; from the left\u0026lt;br\u0026gt; to add to Shopping Cart\" class\u003d\"a-button-input\" type\u003d\"submit\" value\u003d\"Add to Cart\" aria-labelledby\u003d\"submit.add-to-cart-ubb-announce\" /\u003e\u003cspan id\u003d\"submit.add-to-cart-ubb-announce\" class\u003d\"a-button-text\" aria-hidden\u003d\"true\"\u003eAdd to Cart\u003c/span\u003e\u003c/span\u003e\u003c/span\u003e \u003c/div\u003e \u003cdiv class\u003d\"a-section a-spacing-none a-text-center\"\u003e \u003cdiv class\u003d\"a-row a-spacing-none\"\u003e \u003cdiv id\u003d\"oneClickSignInUBB\" class\u003d\"a-section a-spacing-none\"\u003e \u003cdiv class\u003d\"a-divider a-divider-break a-spacing-micro\"\u003e \u003ch5\u003e\u003ca href\u003d\"/gp/product/utility/edit-one-click-pref.html?ie\u003dUTF8\u0026amp;query\u003dselectObb%3dused\u0026amp;returnPath\u003d%2fgp%2fproduct%2f1628391340\"\u003e\u003cspan class\u003d\"a-size-mini\"\u003eTurn on 1-Click ordering for this browser\u003c/span\u003e\u003c/a\u003e\u003c/h5\u003e \u003c/div\u003e \u003c/div\u003e \u003c/div\u003e \u003c/div\u003e \u003c/div\u003e \u003c/div\u003e \u003c/div\u003e \u003c/div\u003e \u003c/div\u003e \u003c/div\u003e \u003cdiv id\u003d\"wishlistButtonStack\" class\u003d\"a-button-stack\"\u003e \u003cdiv id\u003d\"add-to-wishlist-button-group\" data-hover\u003d\"\u0026lt;!-- If PartialItemStateWeblab is true then, showing different Add-to-wish-list tool-tip message which is consistent with Add-to-Cart tool tip message. --\u0026gt; To Add to Your List, choose from options to the left\" class\u003d\"a-button-group a-declarative a-spacing-none\" data-action\u003d\"a-button-group\" role\u003d\"radiogroup\"\u003e \u003cspan id\u003d\"wishListMainButton\" class\u003d\"a-button a-button-group-first a-spacing-none\" role\u003d\"radio\" aria-labelledby\u003d\"wishListMainButton-announce\"\u003e\u003cspan class\u003d\"a-button-inner\"\u003e\u003cinput id\u003d\"add-to-wishlist-button-submit\" name\u003d\"submit.add-to-registry.wishlist\" title\u003d\"Add to List\" data-action\u003d\"atwl-splitbutton-main\" data-hover\u003d\"\u0026lt;!-- If PartialItemStateWeblab is true then, showing different Add-to-wish-list tool-tip message which is consistent with Add-to-Cart tool tip message. --\u0026gt; To Add to Your List, choose from options to the left\" class\u003d\"a-button-input a-declarative\" type\u003d\"submit\" aria-labelledby\u003d\"wishListMainButton-announce\" /\u003e\u003cspan id\u003d\"wishListMainButton-announce\" class\u003d\"a-button-text a-text-left\" aria-hidden\u003d\"true\"\u003e Add to List \u003c/span\u003e\u003c/span\u003e\u003c/span\u003e \u003c/div\u003e \u003cdiv id\u003d\"atwl-dd-spinner-holder\" class\u003d\"a-section a-hidden\"\u003e \u003cdiv class\u003d\"a-row a-dropdown\"\u003e \u003cdiv class\u003d\"a-section a-popover-wrapper\"\u003e \u003cdiv class\u003d\"a-section a-text-center a-popover-inner\"\u003e \u003cdiv class\u003d\"a-box a-popover-loading\"\u003e \u003cdiv class\u003d\"a-box-inner\"\u003e \u003c/div\u003e \u003c/div\u003e \u003c/div\u003e \u003c/div\u003e \u003c/div\u003e \u003c/div\u003e \u003cdiv id\u003d\"atwl-dd-error-holder\" class\u003d\"a-section a-hidden\"\u003e \u003cdiv class\u003d\"a-section a-dropdown\"\u003e \u003cdiv class\u003d\"a-section a-popover-wrapper\"\u003e \u003cdiv class\u003d\"a-section a-spacing-base a-padding-base a-text-left a-popover-inner\"\u003e \u003ch3 class\u003d\"a-color-error\"\u003e Sorry, there was a problem. \u003c/h3\u003e \u003cspan\u003e There was an error retrieving your Wish Lists. Please try again. \u003c/span\u003e \u003c/div\u003e \u003c/div\u003e \u003c/div\u003e \u003c/div\u003e \u003cdiv id\u003d\"atwl-dd-unavail-holder\" class\u003d\"a-section a-hidden\"\u003e \u003cdiv class\u003d\"a-section a-dropdown\"\u003e \u003cdiv class\u003d\"a-section a-popover-wrapper\"\u003e \u003cdiv class\u003d\"a-section a-spacing-base a-padding-base a-text-left a-popover-inner\"\u003e \u003ch3 class\u003d\"a-color-error\"\u003e Sorry, there was a problem. \u003c/h3\u003e \u003cspan\u003e List unavailable. \u003c/span\u003e \u003c/div\u003e \u003c/div\u003e \u003c/div\u003e \u003c/div\u003e \u003cscript type\u003d\"a-state\" data-a-state\u003d\"{\u0026quot;key\u0026quot;:\u0026quot;atwl\u0026quot;}\"\u003e{\"hzPopover\":true,\"wishlistButtonId\":\"add-to-wishlist-button\",\"dropDownHtml\":\"\",\"wishlistButtonSubmitId\":\"add-to-wishlist-button-submit\",\"maxAjaxFailureCount\":\"3\",\"asin\":\"1628391340\"}\u003c/script\u003e \u003cscript type\u003d\"a-state\" data-a-state\u003d\"{\u0026quot;key\u0026quot;:\u0026quot;jumpstart-success-modal-state\u0026quot;}\"\u003e{\"showDPHeadJavaScript\":false,\"showBabyRegSuccessModal\":false,\"addSuccessModalHeader\":\"Added to your Baby Registry\",\"hasBabyReg\":false}\u003c/script\u003e \u003cinput type\u003d\"hidden\" name\u003d\"\" value\u003d\"true\" id\u003d\"is-fully-selected-baby-reg\" /\u003e \u003c/div\u003e \u003cscript type\u003d\"a-state\" data-a-state\u003d\"{\u0026quot;key\u0026quot;:\u0026quot;popoverState\u0026quot;}\"\u003e{\"formId\":\"addToCart\",\"showWishListDropDown\":false,\"babyRegistryId\":\"add-to-registry-baby-button\",\"showBabyRegDropdown\":false,\"wishlistPopoverWidth\":232,\"isAddToWishListDropDownAuiEnabled\":true,\"showPopover\":false,\"isBabyRegistryBrowseScriptDeprecated\":true}\u003c/script\u003e \u003c/form\u003e \u003c/div\u003e \u003c/div\u003e"],"defaultValue":"","isMetaAttribute":false,"extractedValuesCount":1},{"name":"BREADCRUMB","dataType":"STRING","values":["\u003cdiv id\u003d\"wayfinding-breadcrumbs_container\" class\u003d\"a-section a-spacing-none a-padding-medium\"\u003e \u003cdiv id\u003d\"wayfinding-breadcrumbs_feature_div\" class\u003d\"a-subheader a-breadcrumb feature\" data-feature-name\u003d\"wayfinding-breadcrumbs\"\u003e \u003cul class\u003d\"a-unordered-list a-horizontal a-size-small\"\u003e \u003cli\u003e\u003cspan class\u003d\"a-list-item\"\u003e \u003ca class\u003d\"a-link-normal a-color-tertiary\" href\u003d\"/books-used-books-textbooks/b/ref\u003ddp_bc_1/154-5469300-3928621?ie\u003dUTF8\u0026amp;node\u003d283155\"\u003e Books \u003c/a\u003e \u003c/span\u003e\u003c/li\u003e \u003cli class\u003d\"a-breadcrumb-divider\"\u003e\u003cspan class\u003d\"a-list-item a-color-tertiary\"\u003e › \u003c/span\u003e\u003c/li\u003e \u003cli\u003e\u003cspan class\u003d\"a-list-item\"\u003e \u003ca class\u003d\"a-link-normal a-color-tertiary\" href\u003d\"/Christianity-Religion-Spirituality-Books/b/ref\u003ddp_bc_2/154-5469300-3928621?ie\u003dUTF8\u0026amp;node\u003d12290\"\u003e Christian Books \u0026amp; Bibles \u003c/a\u003e \u003c/span\u003e\u003c/li\u003e \u003cli class\u003d\"a-breadcrumb-divider\"\u003e\u003cspan class\u003d\"a-list-item a-color-tertiary\"\u003e › \u003c/span\u003e\u003c/li\u003e \u003cli\u003e\u003cspan class\u003d\"a-list-item\"\u003e \u003ca class\u003d\"a-link-normal a-color-tertiary\" href\u003d\"/Fiction-Poetry-Christianity-Books/b/ref\u003ddp_bc_3/154-5469300-3928621?ie\u003dUTF8\u0026amp;node\u003d172806\"\u003e Literature \u0026amp; Fiction \u003c/a\u003e \u003c/span\u003e\u003c/li\u003e \u003c/ul\u003e \u003c/div\u003e \u003c/div\u003e"],"defaultValue":"","isMetaAttribute":false,"extractedValuesCount":1},{"name":"PARENT_URL","dataType":"STRING","values":[""],"isMetaAttribute":true,"extractedValuesCount":0},{"name":"HTTP_STATUS","dataType":"STRING","values":["200"],"isMetaAttribute":true,"extractedValuesCount":0},{"name":"CRAWLMETADATA","dataType":"STRING","values":["{\"crawlType\":\"PRODUCT\",\"retailerName\":\"amazon_us\",\"site\":\"US\",\"metaCategId\":267,\"leafCategId\":171228,\"productCrawlMetadata\":{\"retailerProductId\":\"1628391340\",\"referencePrice\":0.0,\"referenceShippingPrice\":0.0,\"selectors\":[]},\"tracker\":{\"crawlRequestCreateTS\":1486483774000}}"],"isMetaAttribute":true,"extractedValuesCount":0},{"name":"CRAWL_TIMESTAMP","dataType":"STRING","values":["1486484044649"],"isMetaAttribute":true,"extractedValuesCount":0},{"name":"URL","dataType":"STRING","values":["https://www.amazon.com/gp/product/1628391340/ref\u003dcm_cr_pr_product_top"],"isMetaAttribute":true,"extractedValuesCount":0}]}]} \ No newline at end of file diff --git a/measure/src/test/scala/org/apache/griffin/measure/algo/batch/BatchAccuracyAlgoTest.scala b/measure/src/test/scala/org/apache/griffin/measure/algo/batch/BatchAccuracyAlgoTest.scala index 6615d1adf..3e1551195 100644 --- a/measure/src/test/scala/org/apache/griffin/measure/algo/batch/BatchAccuracyAlgoTest.scala +++ b/measure/src/test/scala/org/apache/griffin/measure/algo/batch/BatchAccuracyAlgoTest.scala @@ -26,8 +26,8 @@ //import org.apache.griffin.measure.config.params.user._ //import org.apache.griffin.measure.config.reader._ //import org.apache.griffin.measure.config.validator._ -//import org.apache.griffin.measure.connector.direct.DirectDataConnector -//import org.apache.griffin.measure.connector.{DataConnector, DataConnectorFactory} +//import org.apache.griffin.measure.data.connector.direct.DirectDataConnector +//import org.apache.griffin.measure.data.connector.{DataConnector, DataConnectorFactory} //import org.apache.griffin.measure.log.Loggable //import org.apache.griffin.measure.rule.expr._ //import org.apache.griffin.measure.rule.{ExprValueUtil, RuleAnalyzer, RuleFactory} diff --git a/measure/src/test/scala/org/apache/griffin/measure/algo/batch/BatchProfileAlgoTest.scala b/measure/src/test/scala/org/apache/griffin/measure/algo/batch/BatchProfileAlgoTest.scala index e0f500a34..8636d0262 100644 --- a/measure/src/test/scala/org/apache/griffin/measure/algo/batch/BatchProfileAlgoTest.scala +++ b/measure/src/test/scala/org/apache/griffin/measure/algo/batch/BatchProfileAlgoTest.scala @@ -26,8 +26,8 @@ //import org.apache.griffin.measure.config.params.user._ //import org.apache.griffin.measure.config.reader._ //import org.apache.griffin.measure.config.validator._ -//import org.apache.griffin.measure.connector.direct.DirectDataConnector -//import org.apache.griffin.measure.connector.{DataConnector, DataConnectorFactory} +//import org.apache.griffin.measure.data.connector.direct.DirectDataConnector +//import org.apache.griffin.measure.data.connector.{DataConnector, DataConnectorFactory} //import org.apache.griffin.measure.log.Loggable //import org.apache.griffin.measure.rule.expr._ //import org.apache.griffin.measure.rule.{ExprValueUtil, RuleAnalyzer, RuleFactory} diff --git a/measure/src/test/scala/org/apache/griffin/measure/algo/batch/DataFrameSaveTest.scala b/measure/src/test/scala/org/apache/griffin/measure/algo/batch/DataFrameSaveTest.scala index a76712f76..d73e45808 100644 --- a/measure/src/test/scala/org/apache/griffin/measure/algo/batch/DataFrameSaveTest.scala +++ b/measure/src/test/scala/org/apache/griffin/measure/algo/batch/DataFrameSaveTest.scala @@ -25,8 +25,8 @@ //import org.apache.griffin.measure.config.params.user._ //import org.apache.griffin.measure.config.reader._ //import org.apache.griffin.measure.config.validator._ -//import org.apache.griffin.measure.connector.DataConnectorFactory -//import org.apache.griffin.measure.connector.direct.DirectDataConnector +//import org.apache.griffin.measure.data.connector.DataConnectorFactory +//import org.apache.griffin.measure.data.connector.direct.DirectDataConnector //import org.apache.griffin.measure.log.Loggable //import org.apache.griffin.measure.rule.expr._ //import org.apache.griffin.measure.rule.{ExprValueUtil, RuleAnalyzer, RuleFactory} diff --git a/measure/src/test/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgoTest.scala b/measure/src/test/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgoTest.scala index 268462082..9381ab242 100644 --- a/measure/src/test/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgoTest.scala +++ b/measure/src/test/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgoTest.scala @@ -29,8 +29,8 @@ //import org.apache.griffin.measure.config.params.user._ //import org.apache.griffin.measure.config.reader._ //import org.apache.griffin.measure.config.validator._ -//import org.apache.griffin.measure.connector.direct.DirectDataConnector -//import org.apache.griffin.measure.connector.{DataConnector, DataConnectorFactory} +//import org.apache.griffin.measure.data.connector.direct.DirectDataConnector +//import org.apache.griffin.measure.data.connector.{DataConnector, DataConnectorFactory} //import org.apache.griffin.measure.log.Loggable //import org.apache.griffin.measure.persist.{Persist, PersistFactory, PersistType} //import org.apache.griffin.measure.result._ diff --git a/measure/src/test/scala/org/apache/griffin/measure/connector/ConnectorTest.scala b/measure/src/test/scala/org/apache/griffin/measure/data/connector/ConnectorTest.scala similarity index 98% rename from measure/src/test/scala/org/apache/griffin/measure/connector/ConnectorTest.scala rename to measure/src/test/scala/org/apache/griffin/measure/data/connector/ConnectorTest.scala index 2139ff77b..0e7c9f8be 100644 --- a/measure/src/test/scala/org/apache/griffin/measure/connector/ConnectorTest.scala +++ b/measure/src/test/scala/org/apache/griffin/measure/data/connector/ConnectorTest.scala @@ -16,7 +16,7 @@ KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License. */ -package org.apache.griffin.measure.connector +package org.apache.griffin.measure.data.connector import java.util.Date import java.util.concurrent.TimeUnit diff --git a/measure/src/test/scala/org/apache/griffin/measure/process/BatchProcessTest.scala b/measure/src/test/scala/org/apache/griffin/measure/process/BatchProcessTest.scala new file mode 100644 index 000000000..1ac9249ca --- /dev/null +++ b/measure/src/test/scala/org/apache/griffin/measure/process/BatchProcessTest.scala @@ -0,0 +1,146 @@ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ +package org.apache.griffin.measure.process + +import org.apache.griffin.measure.algo.ProcessType +import org.apache.griffin.measure.config.params.env._ +import org.apache.griffin.measure.config.params.user._ +import org.apache.griffin.measure.config.params._ +import org.apache.griffin.measure.config.reader.ParamReaderFactory +import org.apache.griffin.measure.config.validator.AllParamValidator +import org.apache.griffin.measure.log.Loggable +import org.apache.griffin.measure.persist.PersistThreadPool +import org.junit.runner.RunWith +import org.scalatest.junit.JUnitRunner +import org.scalatest.{BeforeAndAfter, FunSuite, Matchers} + +import scala.util.{Failure, Success, Try} + +@RunWith(classOf[JUnitRunner]) +class BatchProcessTest extends FunSuite with Matchers with BeforeAndAfter with Loggable { + + val envFile = "src/test/resources/env-test.json" + val confFile = "src/test/resources/config-test2.json" + + val envFsType = "local" + val userFsType = "local" + + val args = Array(envFile, confFile) + + var allParam: AllParam = _ + + before { + // read param files + val envParam = readParamFile[EnvParam](envFile, envFsType) match { + case Success(p) => p + case Failure(ex) => { + error(ex.getMessage) + sys.exit(-2) + } + } + val userParam = readParamFile[UserParam](confFile, userFsType) match { + case Success(p) => p + case Failure(ex) => { + error(ex.getMessage) + sys.exit(-2) + } + } + allParam = AllParam(envParam, userParam) + + // validate param files + validateParams(allParam) match { + case Failure(ex) => { + error(ex.getMessage) + sys.exit(-3) + } + case _ => { + info("params validation pass") + } + } + } + + test ("batch process") { + val procType = allParam.userParam.procType + val proc: DqProcess = procType match { + case ProcessType.batch() => BatchDqProcess(allParam) + case ProcessType.streaming() => StreamingDqProcess(allParam) + case _ => { + error(s"${procType} is unsupported process type!") + sys.exit(-4) + } + } + + // process init + proc.init match { + case Success(_) => { + info("process init success") + } + case Failure(ex) => { + error(s"process init error: ${ex.getMessage}") + shutdown + sys.exit(-5) + } + } + + // process run + proc.run match { + case Success(_) => { + info("process run success") + } + case Failure(ex) => { + error(s"process run error: ${ex.getMessage}") + + if (proc.retriable) { + throw ex + } else { + shutdown + sys.exit(-5) + } + } + } + + // process end + proc.end match { + case Success(_) => { + info("process end success") + } + case Failure(ex) => { + error(s"process end error: ${ex.getMessage}") + shutdown + sys.exit(-5) + } + } + + shutdown + } + + private def readParamFile[T <: Param](file: String, fsType: String)(implicit m : Manifest[T]): Try[T] = { + val paramReader = ParamReaderFactory.getParamReader(file, fsType) + paramReader.readConfig[T] + } + + private def validateParams(allParam: AllParam): Try[Boolean] = { + val allParamValidator = AllParamValidator() + allParamValidator.validate(allParam) + } + + private def shutdown(): Unit = { + PersistThreadPool.shutdown + } +} diff --git a/measure/src/test/scala/org/apache/griffin/measure/process/JsonParseTest.scala b/measure/src/test/scala/org/apache/griffin/measure/process/JsonParseTest.scala new file mode 100644 index 000000000..9d24c5424 --- /dev/null +++ b/measure/src/test/scala/org/apache/griffin/measure/process/JsonParseTest.scala @@ -0,0 +1,479 @@ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ +package org.apache.griffin.measure.process + +import org.apache.griffin.measure.algo.ProcessType +import org.apache.griffin.measure.config.params._ +import org.apache.griffin.measure.config.params.env._ +import org.apache.griffin.measure.config.params.user._ +import org.apache.griffin.measure.config.reader.ParamReaderFactory +import org.apache.griffin.measure.config.validator.AllParamValidator +import org.apache.griffin.measure.log.Loggable +import org.apache.griffin.measure.persist.PersistThreadPool +import org.apache.griffin.measure.utils.JsonUtil +import org.apache.hadoop.hive.ql.exec.UDF +import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.sql.{Column, DataFrame, Row, SQLContext} +import org.apache.spark.sql.catalyst.expressions.GenericRowWithSchema +import org.apache.spark.sql.expressions.UserDefinedAggregateFunction +import org.apache.spark.sql.hive.HiveContext +import org.apache.spark.sql.types._ +import org.junit.runner.RunWith +import org.scalatest.junit.JUnitRunner +import org.scalatest.{BeforeAndAfter, FunSuite, Matchers} + +import scala.collection.mutable.WrappedArray +import scala.util.{Failure, Success, Try} + +@RunWith(classOf[JUnitRunner]) +class JsonParseTest extends FunSuite with Matchers with BeforeAndAfter with Loggable { + + var sparkContext: SparkContext = _ + var sqlContext: SQLContext = _ + + before { + val conf = new SparkConf().setAppName("test json").setMaster("local[*]") + sparkContext = new SparkContext(conf) + sparkContext.setLogLevel("WARN") +// sqlContext = new HiveContext(sparkContext) + sqlContext = new SQLContext(sparkContext) + } + + test ("json test") { + // 0. prepare data +// val dt = +// """ +// |{"name": "s1", "age": 12, "items": [1, 2, 3], +// |"subs": [{"id": 1, "type": "seed"}, {"id": 2, "type": "frog"}], +// |"inner": {"a": 1, "b": 2}, "jstr": "{\"s1\": \"aaa\", \"s2\": 123}" +// |}""".stripMargin +// val rdd0 = sparkContext.parallelize(Seq(dt)).map(Row(_)) + val rdd0 = sparkContext.textFile("src/test/resources/input.msg").map(Row(_)) + + val vtp = StructField("value", StringType) + val df0 = sqlContext.createDataFrame(rdd0, StructType(Array(vtp))) + df0.registerTempTable("src") + +// val fromJson2Array = (s: String) => { +// JsonUtil.fromJson[Seq[String]](s) +// } +// sqlContext.udf.register("from_json_to_array", fromJson2Array) +// +// val df2 = sqlContext.sql("SELECT explode(from_json_to_array(get_json_object(value, '$.seeds'))) as value FROM src") +// df2.printSchema +// df2.show(10) +// df2.registerTempTable("df2") + + + // 1. read from json string to extracted json row + val readSql = "SELECT value FROM src" + val df = sqlContext.sql(readSql) + val rdd = df.map { row => + row.getAs[String]("value") + } + val df1 = sqlContext.read.json(rdd) + df1.printSchema + df1.show(10) + df1.registerTempTable("df1") + + // 2. extract json array into lines +// val rdd2 = df1.flatMap { row => +// row.getAs[WrappedArray[String]]("seeds") +// } +// val df2 = sqlContext.read.json(rdd2) + val df2 = sqlContext.sql("select explode(seeds) as value from df1") +// val tdf = sqlContext.sql("select name, age, explode(items) as item from df1") +// tdf.registerTempTable("tdf") +// val df2 = sqlContext.sql("select struct(name, age, item) as ttt from tdf") + df2.printSchema + df2.show(10) + df2.registerTempTable("df2") + println(df2.count) + + val sql1 = "SELECT value FROM df2" + val df22 = sqlContext.sql(sql1) + val rdd22 = df22.map { row => + row.getAs[String]("value") + } + import org.apache.spark.sql.functions._ + val df23 = sqlContext.read.json(rdd22) + df23.registerTempTable("df23") +// df23.withColumn("par", monotonicallyIncreasingId) + + val df24 = sqlContext.sql("SELECT url, cast(get_json_object(metadata, '$.tracker.crawlRequestCreateTS') as bigint) as ts FROM df23") + df24.printSchema + df24.show(10) + df24.registerTempTable("df24") + println(df24.count) + +// val df25 = sqlContext.sql("select ") + +// +// // 3. extract json string into row +//// val df3 = sqlContext.sql("select cast(get_json_object(metadata, '$.tracker.crawlRequestCreateTS') as bigint), url from df2") +// val df3 = sqlContext.sql("select cast(get_json_object(get_json_object(value, '$.metadata'), '$.tracker.crawlRequestCreateTS') as bigint), get_json_object(value, '$.url') from df2") +// df3.printSchema() +// df3.show(10) +// println(df3.count) + + + +// val df5 = sqlContext.sql("select get_json_object(value, '$.subs') as subs from src") +// df5.printSchema() +// df5.show(10) +// df5.registerTempTable("df5") +// val rdd5 = df5.map { row => +// row.getAs[String]("subs") +// } +// val df6 = sqlContext.read.json(rdd5) +// df6.printSchema +// df6.show(10) + + // 2. extract json string to row +// val df2 = sqlContext.sql("select jstr from df1") +// val rdd2 = df2.map { row => +// row.getAs[String]("jstr") +// } +// val df22 = sqlContext.read.json(rdd2) +// df22.printSchema +// df22.show(100) +// df22.registerTempTable("df2") +// +// val df23 = sqlContext.sql("select json_tuple(jstr, 's1', 's2') from df1") +// df23.printSchema() +// df23.show(100) + + // 3. extract json array into lines ?? + + // 3. flatmap from json row to json row +// val df3 = sqlContext.sql("select explode(subs) as sub, items from df1") +// df3.printSchema() +// df3.show(10) +// df3.registerTempTable("df3") +// +// val df4 = sqlContext.sql("select explode(items) as item, sub from df3") +// df4.printSchema() +// df4.show(10) + +// sqlContext.udf.register("length", (s: WrappedArray[_]) => s.length) + // + // val df2 = sqlContext.sql("SELECT inner from df1") + // df2.registerTempTable("df2") + // df2.printSchema + // df2.show(100) + +// def children(colname: String, df: DataFrame): Array[DataFrame] = { +// val parent = df.schema.fields.filter(_.name == colname).head +// println(parent) +// val fields: Array[StructField] = parent.dataType match { +// case x: StructType => x.fields +// case _ => Array.empty[StructField] +// } +// fields.map(x => col(s"$colname.${x.name}")) +//// fields.foreach(println) +// } +//// +// children("inner", df2) +// +// df2.select(children("bar", df): _*).printSchema + +// val df3 = sqlContext.sql("select inline(subs) from df1") +// df3.printSchema() +// df3.show(100) + +// val rdd2 = df2.flatMap { row => +// row.getAs[GenericRowWithSchema]("inner") :: Nil +// } +// +// rdd2. + +// val funcs = sqlContext.sql("show functions") +// funcs.printSchema() +// funcs.show(1000) +// +// val desc = sqlContext.sql("describe function inline") +// desc.printSchema() +// desc.show(100) + + // + } + + test ("json test 2") { + val rdd0 = sparkContext.textFile("src/test/resources/output.msg").map(Row(_)) + + val vtp = StructField("value", StringType) + val df0 = sqlContext.createDataFrame(rdd0, StructType(Array(vtp))) + df0.registerTempTable("tgt") + +// val fromJson2StringArray = (s: String) => { +// val seq = JsonUtil.fromJson[Seq[Any]](s) +// seq.map(i => JsonUtil.toJson(i)) +// } +// sqlContext.udf.register("from_json_to_string_array", fromJson2StringArray) +// +// val df2 = sqlContext.sql("SELECT from_json_to_string_array(get_json_object(value, '$.groups[0].attrsList')) as value FROM tgt") +// df2.printSchema() +// df2.show(10) +// df2.registerTempTable("df2") +// +// val indexOfStringArray = (sa: String, ) + + + // 1. read from json string to extracted json row + val readSql = "SELECT value FROM tgt" + val df = sqlContext.sql(readSql) + val rdd = df.map { row => + row.getAs[String]("value") + } + val df1 = sqlContext.read.json(rdd) + df1.printSchema + df1.show(10) + df1.registerTempTable("df1") + + + val df2 = sqlContext.sql("select groups[0].attrsList as attrs from df1") + df2.printSchema + df2.show(10) + df2.registerTempTable("df2") + println(df2.count) + + val indexOf = (arr: Seq[String], v: String) => { + arr.indexOf(v) + } + sqlContext.udf.register("index_of", indexOf) + + val df3 = sqlContext.sql("select attrs.values[index_of(attrs.name, 'URL')][0] as url, cast(get_json_object(attrs.values[index_of(attrs.name, 'CRAWLMETADATA')][0], '$.tracker.crawlRequestCreateTS') as bigint) as ts from df2") + df3.printSchema() + df3.show(10) + df3.registerTempTable("df3") + } + + test ("testing") { + val dt = + """ + |{"name": "age", "age": 12, "items": [1, 2, 3], + |"subs": [{"id": 1, "type": "seed"}, {"id": 2, "type": "frog"}], + |"inner": {"a": 1, "b": 2}, "jstr": "{\"s1\": \"aaa\", \"s2\": 123}", "b": true + |}""".stripMargin + val rdd = sparkContext.parallelize(Seq(dt)).map(Row(_)) + val vtp = StructField("value", StringType) + val df = sqlContext.createDataFrame(rdd, StructType(Array(vtp))) + df.registerTempTable("df") + + val df1 = sqlContext.read.json(sqlContext.sql("select * from df").map(r => r.getAs[String]("value"))) + df1.printSchema() + df1.show(10) + df1.registerTempTable("df1") + + val test = (s: String) => { + s.toInt + } + sqlContext.udf.register("to_int", test) + + val df2 = sqlContext.sql("select (b) as aa, inner.a from df1 where age = to_int(\"12\")") + df2.printSchema() + df2.show(10) + } + + test ("test input only sql") { + val rdd0 = sparkContext.textFile("src/test/resources/input.msg").map(Row(_)) + + val vtp = StructField("value", StringType) + val df0 = sqlContext.createDataFrame(rdd0, StructType(Array(vtp))) + df0.registerTempTable("src") + df0.show(10) + + // 1. read from json string to extracted json row + val df1 = sqlContext.sql("SELECT get_json_object(value, '$.seeds') as seeds FROM src") + df1.printSchema + df1.show(10) + df1.registerTempTable("df1") + + val json2StringArray: (String) => Seq[String] = (s: String) => { + val seq = JsonUtil.fromJson[Seq[String]](s) +// seq.map(i => JsonUtil.toJson(i)) + seq + } + sqlContext.udf.register("json_to_string_array", json2StringArray) + + val df2 = sqlContext.sql("SELECT explode(json_to_string_array(seeds)) as seed FROM df1") + df2.printSchema + df2.show(10) + df2.registerTempTable("df2") + + + val df3 = sqlContext.sql("SELECT get_json_object(seed, '$.url') as url, cast(get_json_object(get_json_object(seed, '$.metadata'), '$.tracker.crawlRequestCreateTS') as bigint) as ts FROM df2") + df3.printSchema + df3.show(10) + } + + test ("test output only sql") { + val rdd0 = sparkContext.textFile("src/test/resources/output.msg").map(Row(_)) + + val vtp = StructField("value", StringType) + val df0 = sqlContext.createDataFrame(rdd0, StructType(Array(vtp))) + df0.registerTempTable("tgt") + df0.printSchema() + df0.show(10) + + val json2StringArray: (String) => Seq[String] = (s: String) => { + JsonUtil.fromJson[Seq[String]](s) + } + sqlContext.udf.register("json_to_string_array", json2StringArray) + + val json2StringJsonArray: (String) => Seq[String] = (s: String) => { + val seq = JsonUtil.fromJson[Seq[Any]](s) + seq.map(i => JsonUtil.toJson(i)) + } + sqlContext.udf.register("json_to_string_json_array", json2StringJsonArray) + + val indexOf = (arr: Seq[String], v: String) => { + arr.indexOf(v) + } + sqlContext.udf.register("index_of", indexOf) + + val indexOfField = (arr: Seq[String], k: String, v: String) => { + val seq = arr.flatMap { item => + JsonUtil.fromJson[Map[String, Any]](item).get(k) + } + seq.indexOf(v) + } + sqlContext.udf.register("index_of_field", indexOfField) + + // 1. read from json string to extracted json row + val df1 = sqlContext.sql("SELECT get_json_object(value, '$.groups[0].attrsList') as attrs FROM tgt") + df1.printSchema + df1.show(10) + df1.registerTempTable("df1") + + val df2 = sqlContext.sql("SELECT json_to_string_json_array(attrs) as attrs FROM df1") + df2.printSchema() + df2.show(10) + df2.registerTempTable("df2") + + val df3 = sqlContext.sql("SELECT attrs[index_of_field(attrs, 'name', 'URL')] as attr1, attrs[index_of_field(attrs, 'name', 'CRAWLMETADATA')] as attr2 FROM df2") + df3.printSchema() + df3.show(10) + df3.registerTempTable("df3") + + val df4 = sqlContext.sql("SELECT json_to_string_array(get_json_object(attr1, '$.values'))[0], cast(get_json_object(json_to_string_array(get_json_object(attr2, '$.values'))[0], '$.tracker.crawlRequestCreateTS') as bigint) FROM df3") + df4.printSchema() + df4.show(10) + } + + test ("test from json") { + val fromJson2Map = (str: String) => { + val a = JsonUtil.fromJson[Map[String, Any]](str) + a.mapValues { v => + v match { + case t: String => t + case _ => JsonUtil.toJson(v) + } + } + } + sqlContext.udf.register("from_json_to_map", fromJson2Map) + + val fromJson2Array = (str: String) => { + val a = JsonUtil.fromJson[Seq[Any]](str) + a.map { v => + v match { + case t: String => t + case _ => JsonUtil.toJson(v) + } + } + } + sqlContext.udf.register("from_json_to_array", fromJson2Array) + + // ======================== + + val srdd = sparkContext.textFile("src/test/resources/input.msg").map(Row(_)) + val svtp = StructField("value", StringType) + val sdf0 = sqlContext.createDataFrame(srdd, StructType(Array(svtp))) + sdf0.registerTempTable("sdf0") + sdf0.show(10) + + // 1. read from json string to extracted json row + val sdf1 = sqlContext.sql("SELECT explode(from_json_to_array(get_json_object(value, '$.seeds'))) as seed FROM sdf0") + sdf1.printSchema + sdf1.show(10) + sdf1.registerTempTable("sdf1") + + val sdf2 = sqlContext.sql("SELECT get_json_object(seed, '$.url') as url, cast(get_json_object(get_json_object(seed, '$.metadata'), '$.tracker.crawlRequestCreateTS') as bigint) as ts FROM sdf1") + sdf2.printSchema + sdf2.show(10) + + // --------------------------------------- + + val trdd = sparkContext.textFile("src/test/resources/output.msg").map(Row(_)) + val tvtp = StructField("value", StringType) + val tdf0 = sqlContext.createDataFrame(trdd, StructType(Array(tvtp))) + tdf0.registerTempTable("tdf0") + tdf0.printSchema() + tdf0.show(10) + +// val json2StringArray: (String) => Seq[String] = (s: String) => { +// JsonUtil.fromJson[Seq[String]](s) +// } +// sqlContext.udf.register("json_to_string_array", json2StringArray) +// +// val json2StringJsonArray: (String) => Seq[String] = (s: String) => { +// val seq = JsonUtil.fromJson[Seq[Any]](s) +// seq.map(i => JsonUtil.toJson(i)) +// } +// sqlContext.udf.register("json_to_string_json_array", json2StringJsonArray) +// +// val indexOf = (arr: Seq[String], v: String) => { +// arr.indexOf(v) +// } +// sqlContext.udf.register("index_of", indexOf) +// + val indexOfField = (arr: Seq[String], k: String, v: String) => { + val seq = arr.flatMap { item => + JsonUtil.fromJson[Map[String, Any]](item).get(k) + } + seq.indexOf(v) + } + sqlContext.udf.register("index_of_field", indexOfField) + + // 1. read from json string to extracted json row +// val df1 = sqlContext.sql("SELECT get_json_object(value, '$.groups[0].attrsList') as attrs FROM tdf0") + val tdf1 = sqlContext.sql("SELECT from_json_to_array(get_json_object(value, '$.groups[0].attrsList')) as attrs FROM tdf0") + tdf1.printSchema + tdf1.show(10) + tdf1.registerTempTable("tdf1") + +// val tdf2 = sqlContext.sql("SELECT attrs[index_of_field(attrs, 'name', 'URL')] as attr1, attrs[index_of_field(attrs, 'name', 'CRAWLMETADATA')] as attr2 FROM tdf1") +// tdf2.printSchema() +// tdf2.show(10) +// tdf2.registerTempTable("tdf2") + + val tdf3 = sqlContext.sql("SELECT from_json_to_array(get_json_object(attrs[index_of_field(attrs, 'name', 'URL')], '$.values'))[0] as url, cast(get_json_object(from_json_to_array(get_json_object(attrs[index_of_field(attrs, 'name', 'CRAWLMETADATA')], '$.values'))[0], '$.tracker.crawlRequestCreateTS') as bigint) as ts FROM tdf1") + tdf3.printSchema() + tdf3.show(10) + } + + test ("sql functions") { + val functions = sqlContext.sql("show functions") + functions.printSchema() + functions.show(10) + + val functionNames = functions.map(_.getString(0)).collect + functionNames.foreach(println) + } +} \ No newline at end of file diff --git a/measure/src/test/scala/org/apache/griffin/measure/process/JsonToStructs.scala b/measure/src/test/scala/org/apache/griffin/measure/process/JsonToStructs.scala new file mode 100644 index 000000000..394917c5e --- /dev/null +++ b/measure/src/test/scala/org/apache/griffin/measure/process/JsonToStructs.scala @@ -0,0 +1,85 @@ +package org.apache.griffin.measure.process + +import org.apache.griffin.measure.utils.JsonUtil +import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.analysis.TypeCheckResult +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback +import org.apache.spark.sql.catalyst.util.{ArrayBasedMapData, GenericArrayData} +import org.apache.spark.sql.execution.datasources.json.JSONOptions +import org.apache.spark.sql.types._ +import org.apache.spark.unsafe.types.UTF8String + + +case class JsonToStructs( +// schema: DataType, +// options: Map[String, String], + child: Expression) + extends UnaryExpression with CodegenFallback with ExpectsInputTypes { + override def nullable: Boolean = true + +// def this(schema: DataType, options: Map[String, String], child: Expression) = +// this(schema, options, child, None) + + // Used in `FunctionRegistry` +// def this(child: Expression, schema: Expression) = +// this( +// schema = JsonExprUtils.validateSchemaLiteral(schema), +// options = Map.empty[String, String], +// child = child, +// timeZoneId = None) +// +// def this(child: Expression, schema: Expression, options: Expression) = +// this( +// schema = JsonExprUtils.validateSchemaLiteral(schema), +// options = JsonExprUtils.convertToMapData(options), +// child = child, +// timeZoneId = None) +// +// override def checkInputDataTypes(): TypeCheckResult = schema match { +// case _: StructType | ArrayType(_: StructType, _) => +// super.checkInputDataTypes() +// case _ => TypeCheckResult.TypeCheckFailure( +// s"Input schema ${schema.simpleString} must be a struct or an array of structs.") +// } + + override def dataType: DataType = MapType(StringType, StringType) + +// override def withTimeZone(timeZoneId: String): TimeZoneAwareExpression = +// copy(timeZoneId = Option(timeZoneId)) + + override def nullSafeEval(json: Any): Any = { + if (json.toString.trim.isEmpty) return null + + try { + JsonUtil.fromJson[Map[String, Any]](json.toString) + } catch { + case _: Throwable => null + } + } + + override def inputTypes: Seq[DataType] = StringType :: Nil +} +// +//object JsonExprUtils { +// +// def validateSchemaLiteral(exp: Expression): StructType = exp match { +// case Literal(s, StringType) => CatalystSqlParser.parseTableSchema(s.toString) +// case e => throw new AnalysisException(s"Expected a string literal instead of $e") +// } +// +// def convertToMapData(exp: Expression): Map[String, String] = exp match { +// case m: CreateMap +// if m.dataType.acceptsType(MapType(StringType, StringType, valueContainsNull = false)) => +// val arrayMap = m.eval().asInstanceOf[ArrayBasedMapData] +// ArrayBasedMapData.toScalaMap(arrayMap).map { case (key, value) => +// key.toString -> value.toString +// } +// case m: CreateMap => +// throw new AnalysisException( +// s"A type of keys and values in map() must be string, but got ${m.dataType}") +// case _ => +// throw new AnalysisException("Must use a map() function for options") +// } +//} \ No newline at end of file diff --git a/measure/src/test/scala/org/apache/griffin/measure/rules/dsl/parser/BasicParserTest.scala b/measure/src/test/scala/org/apache/griffin/measure/rules/dsl/parser/BasicParserTest.scala new file mode 100644 index 000000000..ab6f8fb3c --- /dev/null +++ b/measure/src/test/scala/org/apache/griffin/measure/rules/dsl/parser/BasicParserTest.scala @@ -0,0 +1,209 @@ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ +package org.apache.griffin.measure.rules.dsl.parser + +import org.apache.griffin.measure.rules.dsl.expr._ +import org.junit.runner.RunWith +import org.scalatest.junit.JUnitRunner +import org.scalatest.{BeforeAndAfter, FunSuite, Matchers} +//import org.scalatest.FlatSpec +//import org.scalamock.scalatest.MockFactory + +@RunWith(classOf[JUnitRunner]) +class BasicParserTest extends FunSuite with Matchers with BeforeAndAfter { + + val parser = new BasicParser{ + val dataSourceNames: Seq[String] = "source" :: "target" :: Nil + val functionNames: Seq[String] = "func" :: "get_json_object" :: Nil + } + + test("test literal") { + val rule1 = """null""" + val result1 = parser.parseAll(parser.literal, rule1) + result1.successful should be (true) + result1.get.desc should be ("NULL") + + val rule2 = """nan""" + val result2 = parser.parseAll(parser.literal, rule2) + result2.successful should be (true) + result2.get.desc should be ("NaN") + + val rule3 = """'testing'""" + val result3 = parser.parseAll(parser.literal, rule3) + result3.successful should be (true) + result3.get.desc should be ("'testing'") + + val rule4 = """"test again"""" + val result4 = parser.parseAll(parser.literal, rule4) + result4.successful should be (true) + result4.get.desc should be ("\"test again\"") + + val rule5 = """-1.342""" + val result5 = parser.parseAll(parser.literal, rule5) + result5.successful should be (true) + result5.get.desc should be ("-1.342") + + val rule51 = """33""" + val result51 = parser.parseAll(parser.literal, rule51) + result51.successful should be (true) + result51.get.desc should be ("33") + + val rule6 = """2h""" + val result6 = parser.parseAll(parser.literal, rule6) + result6.successful should be (true) + result6.get.desc should be (s"${2 * 3600 * 1000}") + + val rule7 = """true""" + val result7 = parser.parseAll(parser.literal, rule7) + result7.successful should be (true) + result7.get.desc should be ("true") + } + + test ("test selection") { + val rule1 = """source""" + val result1 = parser.parseAll(parser.selection, rule1) + result1.successful should be (true) + result1.get.desc should be ("source") + result1.get.alias should be (Some("source")) + + val rule2 = """source_not_registered""" + val result2 = parser.parseAll(parser.selection, rule2) + result2.successful should be (false) + + val rule3 = """source[12].age""" + val result3 = parser.parseAll(parser.selection, rule3) + result3.successful should be (true) + result3.get.desc should be ("source[12].age") + result3.get.alias should be (Some("age")) + + val rule4 = """source.name.func(target.name)""" + val result4 = parser.parseAll(parser.selection, rule4) + result4.successful should be (true) + result4.get.desc should be ("func(source.name, target.name)") + } + + test ("test math") { + val rule1 = """-1""" + val result1 = parser.parseAll(parser.mathExpression, rule1) + result1.successful should be (true) + result1.get.desc should be ("(-1)") + + val rule2 = "1 + 1" + val result2 = parser.parseAll(parser.mathExpression, rule2) + result2.successful should be (true) + result2.get.desc should be ("1 + 1") + + val rule3 = "source.age + 2 * 5 + target.offset" + val result3 = parser.parseAll(parser.mathExpression, rule3) + result3.successful should be (true) + result3.get.desc should be ("source.age + 2 * 5 + target.offset") + + val rule4 = "(source.age + 2) * (5 + target.offset)" + val result4 = parser.parseAll(parser.mathExpression, rule4) + result4.successful should be (true) + result4.get.desc should be ("(source.age + 2) * (5 + target.offset)") + } + + test ("test logical") { + val rule1 = "source.age in (12 + 3, 23, 34)" + val result1 = parser.parseAll(parser.logicalExpression, rule1) + result1.successful should be (true) + result1.get.desc should be ("source.age IN (12 + 3, 23, 34)") + + val rule2 = "source.age between (12 + 3, 23, 34)" + val result2 = parser.parseAll(parser.logicalExpression, rule2) + result2.successful should be (true) + result2.get.desc should be ("source.age BETWEEN 12 + 3 AND 23") + + val rule3 = "source.age between (12 + 3)" + val result3 = parser.parseAll(parser.logicalExpression, rule3) + result3.successful should be (true) + assertThrows[Exception](result3.get.desc) + + val rule4 = "source.name like '%tk'" + val result4 = parser.parseAll(parser.logicalExpression, rule4) + result4.successful should be (true) + result4.get.desc should be ("source.name LIKE '%tk'") + + val rule5 = "source.desc is not null" + val result5 = parser.parseAll(parser.logicalExpression, rule5) + result5.successful should be (true) + result5.get.desc should be ("source.desc IS NOT NULL") + + val rule6 = "source.desc is not nan" + val result6 = parser.parseAll(parser.logicalExpression, rule6) + result6.successful should be (true) + result6.get.desc should be ("NOT isnan(source.desc)") + + val rule7 = "!source.ok and source.name = target.name && (source.age between 12 and 52) && target.desc is not null" + val result7 = parser.parseAll(parser.logicalExpression, rule7) + result7.successful should be (true) + result7.get.desc should be ("(NOT source.ok) AND source.name = target.name AND (source.age BETWEEN 12 AND 52) AND target.desc IS NOT NULL") + + val rule8 = "!(10 != 30 and !(31 > 2) or (45 <= 8 and 33 <> 0))" + val result8 = parser.parseAll(parser.logicalExpression, rule8) + result8.successful should be (true) + result8.get.desc should be ("(NOT (10 != 30 AND (NOT (31 > 2)) OR (45 <= 8 AND 33 <> 0)))") + + } + + test ("test expression") { + val rule3 = "source.age + 2 * 5 + target.offset" + val result3 = parser.parseAll(parser.expression, rule3) + result3.successful should be (true) + result3.get.desc should be ("source.age + 2 * 5 + target.offset") + + val rule4 = "(source.age + 2) * (5 + target.offset)" + val result4 = parser.parseAll(parser.expression, rule4) + result4.successful should be (true) + result4.get.desc should be ("(source.age + 2) * (5 + target.offset)") + + val rule7 = "!source.ok and source.name = target.name && (source.age between 12 and 52) && target.desc is not null" + val result7 = parser.parseAll(parser.expression, rule7) + result7.successful should be (true) + result7.get.desc should be ("(NOT source.ok) AND source.name = target.name AND (source.age BETWEEN 12 AND 52) AND target.desc IS NOT NULL") + + val rule8 = "!(10 != 30 and !(31 > 2) or (45 <= 8 and 33 <> 0))" + val result8 = parser.parseAll(parser.expression, rule8) + result8.successful should be (true) + result8.get.desc should be ("(NOT (10 != 30 AND (NOT (31 > 2)) OR (45 <= 8 AND 33 <> 0)))") + + val rule1 = "source.user_id = target.user_id AND source.first_name = 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" + val result1 = parser.parseAll(parser.expression, rule1) + result1.successful should be (true) + result1.get.desc should be ("source.user_id = target.user_id AND source.first_name = 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") + } + + test ("test function") { + val rule3 = "source.age + 2 * 5 + target.offset * func('a', source.name)" + val result3 = parser.parseAll(parser.expression, rule3) + result3.successful should be (true) + result3.get.desc should be ("source.age + 2 * 5 + target.offset * func('a', source.name)") + result3.get.alias should be (None) + } + + test ("test alias") { + val rule1 = "(source.name as name)" + val result1 = parser.parseAll(parser.expression, rule1) + result1.successful should be (true) + result1.get.desc should be ("source.name") + result1.get.alias should be (Some("name")) + } + +} From 64b493e15a82b5fd1b2da780bf9c4d966ed80b83 Mon Sep 17 00:00:00 2001 From: Lionel Liu Date: Fri, 25 Aug 2017 17:03:55 +0800 Subject: [PATCH 052/111] del alias --- .../rules/dsl/analyzer/AccuracyAnalyzer.scala | 6 ++--- .../rules/dsl/analyzer/BasicAnalyzer.scala | 11 +++----- .../measure/rules/dsl/expr/AliasExpr.scala | 27 ------------------- .../griffin/measure/rules/dsl/expr/Expr.scala | 2 -- .../measure/rules/dsl/expr/FunctionExpr.scala | 1 - .../measure/rules/dsl/expr/LiteralExpr.scala | 1 - .../measure/rules/dsl/expr/LogicalExpr.scala | 2 -- .../measure/rules/dsl/expr/MathExpr.scala | 2 -- .../measure/rules/dsl/expr/SelectExpr.scala | 18 +++++++------ .../rules/dsl/parser/BasicParser.scala | 17 ++++-------- .../rules/dsl/parser/BasicParserTest.scala | 14 +--------- 11 files changed, 22 insertions(+), 79 deletions(-) delete mode 100644 measure/src/main/scala/org/apache/griffin/measure/rules/dsl/expr/AliasExpr.scala diff --git a/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/analyzer/AccuracyAnalyzer.scala b/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/analyzer/AccuracyAnalyzer.scala index 6ef690f28..bc5403015 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/analyzer/AccuracyAnalyzer.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/analyzer/AccuracyAnalyzer.scala @@ -18,7 +18,7 @@ under the License. */ package org.apache.griffin.measure.rules.dsl.analyzer -import org.apache.griffin.measure.rules.dsl.expr.Expr +import org.apache.griffin.measure.rules.dsl.expr._ case class AccuracyAnalyzer(expr: Expr, sourceName: String, targetName: String) extends BasicAnalyzer { @@ -27,11 +27,11 @@ case class AccuracyAnalyzer(expr: Expr, sourceName: String, targetName: String) val sourceSelectionExprs = { val seq = seqSelectionExprs(sourceName) - expr.preOrderTraverseDepthFirst(Seq[Expr]())(seq, combSelectionExprs) + expr.preOrderTraverseDepthFirst(Seq[SelectionExpr]())(seq, combSelectionExprs) } val targetSelectionExprs = { val seq = seqSelectionExprs(targetName) - expr.preOrderTraverseDepthFirst(Seq[Expr]())(seq, combSelectionExprs) + expr.preOrderTraverseDepthFirst(Seq[SelectionExpr]())(seq, combSelectionExprs) } } diff --git a/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/analyzer/BasicAnalyzer.scala b/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/analyzer/BasicAnalyzer.scala index 2ab2a5bdb..21192c79d 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/analyzer/BasicAnalyzer.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/analyzer/BasicAnalyzer.scala @@ -33,17 +33,12 @@ trait BasicAnalyzer extends Serializable { } val combDataSourceNames = (a: Set[String], b: Set[String]) => a ++ b - val seqSelectionExprs = (dsName: String) => (expr: Expr, v: Seq[Expr]) => { + val seqSelectionExprs = (dsName: String) => (expr: Expr, v: Seq[SelectionExpr]) => { expr match { - case se @ SelectionExpr(head: DataSourceHeadExpr, _) => { - head.alias match { - case Some(a) if (a == dsName) => v :+ se - case _ => v - } - } + case se @ SelectionExpr(head: DataSourceHeadExpr, _, _) if (head.desc == dsName) => v :+ se case _ => v } } - val combSelectionExprs = (a: Seq[Expr], b: Seq[Expr]) => a ++ b + val combSelectionExprs = (a: Seq[SelectionExpr], b: Seq[SelectionExpr]) => a ++ b } diff --git a/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/expr/AliasExpr.scala b/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/expr/AliasExpr.scala deleted file mode 100644 index eba1db3b2..000000000 --- a/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/expr/AliasExpr.scala +++ /dev/null @@ -1,27 +0,0 @@ -/* -Licensed to the Apache Software Foundation (ASF) under one -or more contributor license agreements. See the NOTICE file -distributed with this work for additional information -regarding copyright ownership. The ASF licenses this file -to you under the Apache License, Version 2.0 (the -"License"); you may not use this file except in compliance -with the License. You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, -software distributed under the License is distributed on an -"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -KIND, either express or implied. See the License for the -specific language governing permissions and limitations -under the License. -*/ -package org.apache.griffin.measure.rules.dsl.expr - -case class AliasExpr(expr: Expr, aliasName: String) extends Expr { - - addChild(expr) - - def desc: String = expr.desc - def alias: Option[String] = Some(aliasName) -} \ No newline at end of file diff --git a/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/expr/Expr.scala b/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/expr/Expr.scala index 08406fa48..b789a3ccf 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/expr/Expr.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/expr/Expr.scala @@ -22,6 +22,4 @@ trait Expr extends TreeNode with Serializable { def desc: String - def alias: Option[String] - } diff --git a/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/expr/FunctionExpr.scala b/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/expr/FunctionExpr.scala index 19af690b2..408b46627 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/expr/FunctionExpr.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/expr/FunctionExpr.scala @@ -25,5 +25,4 @@ case class FunctionExpr(functionName: String, args: Seq[Expr]) extends Expr { def desc: String = { s"${functionName}(${args.map(_.desc).mkString(", ")})" } - def alias: Option[String] = None } \ No newline at end of file diff --git a/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/expr/LiteralExpr.scala b/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/expr/LiteralExpr.scala index 96532edae..375b5f52d 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/expr/LiteralExpr.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/expr/LiteralExpr.scala @@ -21,7 +21,6 @@ package org.apache.griffin.measure.rules.dsl.expr import org.apache.griffin.measure.utils.TimeUtil trait LiteralExpr extends Expr { - def alias: Option[String] = None } case class LiteralNullExpr(str: String) extends LiteralExpr { diff --git a/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/expr/LogicalExpr.scala b/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/expr/LogicalExpr.scala index 155d2f274..6aada8361 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/expr/LogicalExpr.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/expr/LogicalExpr.scala @@ -19,7 +19,6 @@ under the License. package org.apache.griffin.measure.rules.dsl.expr trait LogicalExpr extends Expr { - def alias: Option[String] = None } case class InExpr(head: Expr, is: Boolean, range: Seq[Expr]) extends LogicalExpr { @@ -88,7 +87,6 @@ case class LogicalFactorExpr(factor: Expr, withBracket: Boolean) extends Logical def desc: String = { if (withBracket) s"(${factor.desc})" else factor.desc } - override def alias: Option[String] = factor.alias } case class UnaryLogicalExpr(oprs: Seq[String], factor: LogicalExpr) extends LogicalExpr { diff --git a/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/expr/MathExpr.scala b/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/expr/MathExpr.scala index 12c51d8ff..3af54b482 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/expr/MathExpr.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/expr/MathExpr.scala @@ -19,7 +19,6 @@ under the License. package org.apache.griffin.measure.rules.dsl.expr trait MathExpr extends Expr { - def alias: Option[String] = None } case class MathFactorExpr(factor: Expr, withBracket: Boolean) extends MathExpr { @@ -29,7 +28,6 @@ case class MathFactorExpr(factor: Expr, withBracket: Boolean) extends MathExpr { def desc: String = { if (withBracket) s"(${factor.desc})" else factor.desc } - override def alias: Option[String] = factor.alias } case class UnaryMathExpr(oprs: Seq[String], factor: MathExpr) extends MathExpr { diff --git a/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/expr/SelectExpr.scala b/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/expr/SelectExpr.scala index d8066dbe7..7eb571415 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/expr/SelectExpr.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/expr/SelectExpr.scala @@ -24,7 +24,6 @@ trait HeadExpr extends Expr { case class DataSourceHeadExpr(name: String) extends HeadExpr { def desc: String = name - def alias: Option[String] = Some(name) } case class OtherHeadExpr(expr: Expr) extends HeadExpr { @@ -32,13 +31,14 @@ case class OtherHeadExpr(expr: Expr) extends HeadExpr { addChild(expr) def desc: String = expr.desc - def alias: Option[String] = expr.alias } // ------------- trait SelectExpr extends Expr { + def alias: Option[String] + } case class FieldSelectExpr(field: String) extends SelectExpr { @@ -51,7 +51,7 @@ case class IndexSelectExpr(index: Expr) extends SelectExpr { addChild(index) def desc: String = s"[${index.desc}]" - def alias: Option[String] = None + def alias: Option[String] = Some(desc) } case class FunctionSelectExpr(functionName: String, args: Seq[Expr]) extends SelectExpr { @@ -64,7 +64,7 @@ case class FunctionSelectExpr(functionName: String, args: Seq[Expr]) extends Sel // ------------- -case class SelectionExpr(head: HeadExpr, selectors: Seq[SelectExpr]) extends Expr { +case class SelectionExpr(head: HeadExpr, selectors: Seq[SelectExpr], aliasOpt: Option[String]) extends SelectExpr { addChildren(head +: selectors) @@ -80,9 +80,11 @@ case class SelectionExpr(head: HeadExpr, selectors: Seq[SelectExpr]) extends Exp } } def alias: Option[String] = { - selectors.lastOption match { - case Some(last) => last.alias - case _ => head.alias - } + if (aliasOpt.isEmpty) { + selectors.lastOption match { + case Some(last) => last.alias + case _ => None + } + } else aliasOpt } } \ No newline at end of file diff --git a/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/parser/BasicParser.scala b/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/parser/BasicParser.scala index bc2de36c4..2d7666f1c 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/parser/BasicParser.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/parser/BasicParser.scala @@ -168,8 +168,8 @@ trait BasicParser extends JavaTokenParsers with Serializable { * ::= */ - def selection: Parser[SelectionExpr] = selectionHead ~ rep(selector) ^^ { - case head ~ sels => SelectionExpr(head, sels) + def selection: Parser[SelectionExpr] = selectionHead ~ rep(selector) ~ opt(asAlias) ^^ { + case head ~ sels ~ aliasOpt => SelectionExpr(head, sels, aliasOpt) } def selectionHead: Parser[HeadExpr] = DataSourceName ^^ { DataSourceHeadExpr(_) } | function ^^ { OtherHeadExpr(_) } def selector: Parser[SelectExpr] = functionSelect | fieldSelect | indexSelect @@ -179,6 +179,8 @@ trait BasicParser extends JavaTokenParsers with Serializable { case _ ~ name ~ _ ~ args ~ _ => FunctionSelectExpr(name, args) } + def asAlias: Parser[String] = AS ~> TableFieldName + /** * -- math expr -- * ::= | | | | "(" ")" @@ -187,7 +189,7 @@ trait BasicParser extends JavaTokenParsers with Serializable { * ::= */ - def mathFactor: Parser[MathExpr] = (literal | aliasExpr | function | selection) ^^ { + def mathFactor: Parser[MathExpr] = (literal | function | selection) ^^ { MathFactorExpr(_, false) } | LBR ~> mathExpression <~ RBR ^^ { MathFactorExpr(_, true) @@ -279,13 +281,4 @@ trait BasicParser extends JavaTokenParsers with Serializable { } def argument: Parser[Expr] = expression - /** - * -- alias expr -- - * = - */ - - def aliasExpr: Parser[AliasExpr] = LBR ~> expression ~ AS ~ TableFieldName <~ RBR ^^ { - case expr ~ _ ~ aname => AliasExpr(expr, aname) - } - } diff --git a/measure/src/test/scala/org/apache/griffin/measure/rules/dsl/parser/BasicParserTest.scala b/measure/src/test/scala/org/apache/griffin/measure/rules/dsl/parser/BasicParserTest.scala index ab6f8fb3c..13f230757 100644 --- a/measure/src/test/scala/org/apache/griffin/measure/rules/dsl/parser/BasicParserTest.scala +++ b/measure/src/test/scala/org/apache/griffin/measure/rules/dsl/parser/BasicParserTest.scala @@ -80,7 +80,6 @@ class BasicParserTest extends FunSuite with Matchers with BeforeAndAfter { val result1 = parser.parseAll(parser.selection, rule1) result1.successful should be (true) result1.get.desc should be ("source") - result1.get.alias should be (Some("source")) val rule2 = """source_not_registered""" val result2 = parser.parseAll(parser.selection, rule2) @@ -132,9 +131,7 @@ class BasicParserTest extends FunSuite with Matchers with BeforeAndAfter { result2.get.desc should be ("source.age BETWEEN 12 + 3 AND 23") val rule3 = "source.age between (12 + 3)" - val result3 = parser.parseAll(parser.logicalExpression, rule3) - result3.successful should be (true) - assertThrows[Exception](result3.get.desc) + assertThrows[Exception](parser.parseAll(parser.logicalExpression, rule3)) val rule4 = "source.name like '%tk'" val result4 = parser.parseAll(parser.logicalExpression, rule4) @@ -195,15 +192,6 @@ class BasicParserTest extends FunSuite with Matchers with BeforeAndAfter { val result3 = parser.parseAll(parser.expression, rule3) result3.successful should be (true) result3.get.desc should be ("source.age + 2 * 5 + target.offset * func('a', source.name)") - result3.get.alias should be (None) - } - - test ("test alias") { - val rule1 = "(source.name as name)" - val result1 = parser.parseAll(parser.expression, rule1) - result1.successful should be (true) - result1.get.desc should be ("source.name") - result1.get.alias should be (Some("name")) } } From 41c7ee8ab53bbfcfb6acf184cd099f5c66f53884 Mon Sep 17 00:00:00 2001 From: Lionel Liu Date: Sun, 27 Aug 2017 20:55:44 +0800 Subject: [PATCH 053/111] batch accu done --- .../measure/process/BatchDqProcess.scala | 10 +-- .../process/engine/DataFrameOprEngine.scala | 40 +++++++++--- .../process/engine/SparkSqlEngine.scala | 5 +- .../rules/adaptor/DataFrameOprAdaptor.scala | 8 ++- .../rules/adaptor/GriffinDslAdaptor.scala | 61 +++++++++++++------ .../measure/rules/adaptor/RuleAdaptor.scala | 12 ++-- .../rules/adaptor/RuleAdaptorGroup.scala | 1 - .../rules/adaptor/SparkSqlAdaptor.scala | 6 ++ .../griffin/measure/rules/dsl/expr/Expr.scala | 2 + .../measure/rules/dsl/expr/FunctionExpr.scala | 5 +- .../measure/rules/dsl/expr/LiteralExpr.scala | 1 + .../measure/rules/dsl/expr/LogicalExpr.scala | 35 ++++++++++- .../measure/rules/dsl/expr/MathExpr.scala | 17 +++++- .../measure/rules/dsl/expr/SelectExpr.scala | 12 ++++ .../measure/rules/step/DfOprStep.scala | 5 +- measure/src/test/resources/config-test1.json | 15 +++-- .../measure/process/JsonParseTest.scala | 23 ++++--- 17 files changed, 194 insertions(+), 64 deletions(-) diff --git a/measure/src/main/scala/org/apache/griffin/measure/process/BatchDqProcess.scala b/measure/src/main/scala/org/apache/griffin/measure/process/BatchDqProcess.scala index 250f84076..752c727f0 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/process/BatchDqProcess.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/process/BatchDqProcess.scala @@ -72,17 +72,17 @@ case class BatchDqProcess(allParam: AllParam) extends DqProcess { // generate rule steps val ruleSteps = RuleAdaptorGroup.genConcreteRuleSteps(userParam.evaluateRuleParam) - // get calculation engine - val engine = DqEngineFactory.genDqEngines(sqlContext, null) + // get dq engines + val dqEngines = DqEngineFactory.genDqEngines(sqlContext, null) // init data sources - engine.initDataSources(userParam.dataSources) + dqEngines.initDataSources(userParam.dataSources) // run rules - engine.runRuleSteps(ruleSteps) + dqEngines.runRuleSteps(ruleSteps) // persist results - engine.persistResults(ruleSteps, persist) + dqEngines.persistResults(ruleSteps, persist) // end time val endTime = new Date().getTime diff --git a/measure/src/main/scala/org/apache/griffin/measure/process/engine/DataFrameOprEngine.scala b/measure/src/main/scala/org/apache/griffin/measure/process/engine/DataFrameOprEngine.scala index 70d3213eb..fc454e385 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/process/engine/DataFrameOprEngine.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/process/engine/DataFrameOprEngine.scala @@ -22,10 +22,11 @@ import org.apache.griffin.measure.config.params.user.DataSourceParam import org.apache.griffin.measure.data.source.{DataSource, DataSourceFactory} import org.apache.griffin.measure.persist.Persist import org.apache.griffin.measure.rules.step._ -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.{DataFrame, SQLContext} import org.apache.spark.streaming.StreamingContext -case class DataFrameOprEngine(sqlContext: SQLContext, @transient ssc: StreamingContext) extends DqEngine { +case class DataFrameOprEngine(sqlContext: SQLContext, @transient ssc: StreamingContext + ) extends DqEngine { def genDataSource(dataSourceParam: DataSourceParam): Option[DataSource] = { DataSourceFactory.genDataSource(sqlContext, ssc, dataSourceParam) @@ -33,15 +34,21 @@ case class DataFrameOprEngine(sqlContext: SQLContext, @transient ssc: StreamingC def runRuleStep(ruleStep: ConcreteRuleStep): Boolean = { ruleStep match { - case DfOprStep(name, rule, _) => { + case DfOprStep(name, rule, details) => { try { -// val rdf = sqlContext.sql(rule) -// rdf.registerTempTable(name) - // fixme + rule match { + case DataFrameOprs._fromJson => { + val df = DataFrameOprs.fromJson(sqlContext, name, details) + df.registerTempTable(name) + } + case _ => { + throw new Exception(s"df opr [ ${rule} ] not supported") + } + } true } catch { case e: Throwable => { - error(s"run rule ${name} error: ${e.getMessage}") + error(s"run df opr [ ${rule} ] error: ${e.getMessage}") false } } @@ -69,6 +76,25 @@ case class DataFrameOprEngine(sqlContext: SQLContext, @transient ssc: StreamingC } +object DataFrameOprs { + + final val _fromJson = "from_json" + + def fromJson(sqlContext: SQLContext, name: String, details: Map[String, Any]): DataFrame = { + val _dfName = "df.name" + val _colName = "col.name" + val dfName = details.getOrElse(_dfName, name).toString + val colNameOpt = details.get(_colName).map(_.toString) + + val df = sqlContext.table(dfName) + val rdd = colNameOpt match { + case Some(colName: String) => df.map(_.getAs[String](colName)) + case _ => df.map(_.getAs[String](0)) + } + sqlContext.read.json(rdd) + } + +} diff --git a/measure/src/main/scala/org/apache/griffin/measure/process/engine/SparkSqlEngine.scala b/measure/src/main/scala/org/apache/griffin/measure/process/engine/SparkSqlEngine.scala index bfefeefef..8c24ba6dd 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/process/engine/SparkSqlEngine.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/process/engine/SparkSqlEngine.scala @@ -26,7 +26,8 @@ import org.apache.griffin.measure.utils.JsonUtil import org.apache.spark.sql.{DataFrame, SQLContext} import org.apache.spark.streaming.StreamingContext -case class SparkSqlEngine(sqlContext: SQLContext, @transient ssc: StreamingContext) extends DqEngine { +case class SparkSqlEngine(sqlContext: SQLContext, @transient ssc: StreamingContext + ) extends DqEngine { def genDataSource(dataSourceParam: DataSourceParam): Option[DataSource] = { DataSourceFactory.genDataSource(sqlContext, ssc, dataSourceParam) @@ -41,7 +42,7 @@ case class SparkSqlEngine(sqlContext: SQLContext, @transient ssc: StreamingConte true } catch { case e: Throwable => { - error(s"run rule ${name} error: ${e.getMessage}") + error(s"run spark sql [ ${rule} ] error: ${e.getMessage}") false } } diff --git a/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/DataFrameOprAdaptor.scala b/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/DataFrameOprAdaptor.scala index e4b75ed1c..65d648d20 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/DataFrameOprAdaptor.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/DataFrameOprAdaptor.scala @@ -23,7 +23,13 @@ import org.apache.griffin.measure.rules.step._ case class DataFrameOprAdaptor() extends RuleAdaptor { def genRuleStep(param: Map[String, Any]): Seq[RuleStep] = { - DfOprStep(getName(param), getRule(param), getPersistType(param)) :: Nil + DfOprStep(getName(param), getRule(param), getDetails(param)) :: Nil + } + def adaptConcreteRuleStep(ruleStep: RuleStep): Seq[ConcreteRuleStep] = { + ruleStep match { + case rs @ DfOprStep(_, _, _) => rs :: Nil + case _ => Nil + } } def getTempSourceNames(param: Map[String, Any]): Seq[String] = { diff --git a/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/GriffinDslAdaptor.scala b/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/GriffinDslAdaptor.scala index 8d617081c..05755a4e4 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/GriffinDslAdaptor.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/GriffinDslAdaptor.scala @@ -47,30 +47,23 @@ case class GriffinDslAdaptor(dataSourceNames: Seq[String], functionNames: Seq[St } nameOpt.getOrElse(key) } - def resultPersistType(param: Map[String, Any], key: String): PersistType = { + def resultPersistType(param: Map[String, Any], key: String, defPersistType: PersistType): PersistType = { param.get(key) match { case Some(prm: Map[String, Any]) => StepInfo.getPersistType(prm) - case _ => NonePersistType + case _ => defPersistType } } } val _dqType = "dq.type" - val _details = "details" + + protected def getDqType(param: Map[String, Any]) = DqType(param.getOrElse(_dqType, "").toString) val filteredFunctionNames = functionNames.filter { fn => fn.matches("""^[a-zA-Z_]\w*$""") } val parser = GriffinDslParser(dataSourceNames, filteredFunctionNames) - protected def getDqType(param: Map[String, Any]) = DqType(param.getOrElse(_dqType, "").toString) - protected def getDetails(param: Map[String, Any]) = { - param.get(_details) match { - case Some(p: Map[String, Any]) => p - case _ => Map[String, Any]() - } - } - def genRuleStep(param: Map[String, Any]): Seq[RuleStep] = { GriffinDslStep(getName(param), getRule(param), getDqType(param), getDetails(param)) :: Nil } @@ -100,7 +93,7 @@ case class GriffinDslAdaptor(dataSourceNames: Seq[String], functionNames: Seq[St } } - override def adaptConcreteRuleStep(ruleStep: RuleStep): Seq[ConcreteRuleStep] = { + def adaptConcreteRuleStep(ruleStep: RuleStep): Seq[ConcreteRuleStep] = { ruleStep match { case rs @ GriffinDslStep(_, rule, _, _) => { val exprOpt = try { @@ -129,13 +122,14 @@ case class GriffinDslAdaptor(dataSourceNames: Seq[String], functionNames: Seq[St } private def transConcreteRuleSteps(ruleStep: GriffinDslStep, expr: Expr): Seq[ConcreteRuleStep] = { + val details = ruleStep.details ruleStep.dqType match { case AccuracyType => { - val sourceName = AccuracyInfo.getNameOpt(ruleStep.details, AccuracyInfo._Source) match { + val sourceName = AccuracyInfo.getNameOpt(details, AccuracyInfo._Source) match { case Some(name) => name case _ => dataSourceNames.head } - val targetName = AccuracyInfo.getNameOpt(ruleStep.details, AccuracyInfo._Target) match { + val targetName = AccuracyInfo.getNameOpt(details, AccuracyInfo._Target) match { case Some(name) => name case _ => dataSourceNames.tail.head } @@ -151,21 +145,48 @@ case class GriffinDslAdaptor(dataSourceNames: Seq[String], functionNames: Seq[St s"${sel.desc}${alias}" }.mkString(", ") - s"SELECT ${selClause} FROM ${sourceName} LEFT JOIN ${targetName}" + val onClause = expr.coalesceDesc + + val sourceIsNull = analyzer.sourceSelectionExprs.map { sel => + s"${sel.desc} IS NULL" + }.mkString(" AND ") + val targetIsNull = analyzer.targetSelectionExprs.map { sel => + s"${sel.desc} IS NULL" + }.mkString(" AND ") + val whereClause = s"(NOT (${sourceIsNull})) AND (${targetIsNull})" + + s"SELECT ${selClause} FROM ${sourceName} LEFT JOIN ${targetName} ON ${onClause} WHERE ${whereClause}" } - println(missRecordSql) + val missRecordName = AccuracyInfo.resultName(details, AccuracyInfo._MissRecord) + val missRecordStep = SparkSqlStep( + missRecordName, + missRecordSql, + AccuracyInfo.resultPersistType(details, AccuracyInfo._MissRecord, RecordPersistType) + ) // 2. miss count val missCountSql = { - "" + s"SELECT COUNT(*) AS `miss.count` FROM `${missRecordName}`" } + val missCountName = AccuracyInfo.resultName(details, AccuracyInfo._MissCount) + val missCountStep = SparkSqlStep( + missCountName, + missCountSql, + AccuracyInfo.resultPersistType(details, AccuracyInfo._MissCount, MetricPersistType) + ) // 3. total count val totalCountSql = { - "" + s"SELECT COUNT(*) AS `total.count` FROM ${sourceName}" } - - Nil + val totalCountName = AccuracyInfo.resultName(details, AccuracyInfo._TotalCount) + val totalCountStep = SparkSqlStep( + totalCountName, + totalCountSql, + AccuracyInfo.resultPersistType(details, AccuracyInfo._TotalCount, MetricPersistType) + ) + + missRecordStep :: missCountStep :: totalCountStep :: Nil } case ProfilingType => { Nil diff --git a/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/RuleAdaptor.scala b/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/RuleAdaptor.scala index 26b6ced62..3641ea499 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/RuleAdaptor.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/RuleAdaptor.scala @@ -31,23 +31,25 @@ trait RuleAdaptor extends Loggable with Serializable { val _name = "name" val _rule = "rule" val _persistType = "persist.type" + val _details = "details" protected def getName(param: Map[String, Any]) = param.getOrElse(_name, RuleStepNameGenerator.genName).toString protected def getRule(param: Map[String, Any]) = param.getOrElse(_rule, "").toString protected def getPersistType(param: Map[String, Any]) = PersistType(param.getOrElse(_persistType, "").toString) + protected def getDetails(param: Map[String, Any]) = param.get(_details) match { + case Some(dt: Map[String, Any]) => dt + case _ => Map[String, Any]() + } def getTempSourceNames(param: Map[String, Any]): Seq[String] def genRuleStep(param: Map[String, Any]): Seq[RuleStep] def genConcreteRuleStep(param: Map[String, Any]): Seq[ConcreteRuleStep] = { genRuleStep(param).flatMap { rs => - rs match { - case s: ConcreteRuleStep => Some(s) - case _ => adaptConcreteRuleStep(rs) - } + adaptConcreteRuleStep(rs) } } - protected def adaptConcreteRuleStep(ruleStep: RuleStep): Seq[ConcreteRuleStep] = Nil + protected def adaptConcreteRuleStep(ruleStep: RuleStep): Seq[ConcreteRuleStep] } diff --git a/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/RuleAdaptorGroup.scala b/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/RuleAdaptorGroup.scala index 8a6b40f5d..3b9f5a5bd 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/RuleAdaptorGroup.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/RuleAdaptorGroup.scala @@ -84,7 +84,6 @@ object RuleAdaptorGroup { } (preSteps ++ curSteps, curNames) } - steps.foreach(println) steps } diff --git a/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/SparkSqlAdaptor.scala b/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/SparkSqlAdaptor.scala index 4ae1129ef..abe01c845 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/SparkSqlAdaptor.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/SparkSqlAdaptor.scala @@ -27,6 +27,12 @@ case class SparkSqlAdaptor() extends RuleAdaptor { def genRuleStep(param: Map[String, Any]): Seq[RuleStep] = { SparkSqlStep(getName(param), getRule(param), getPersistType(param)) :: Nil } + def adaptConcreteRuleStep(ruleStep: RuleStep): Seq[ConcreteRuleStep] = { + ruleStep match { + case rs @ SparkSqlStep(_, _, _) => rs :: Nil + case _ => Nil + } + } def getTempSourceNames(param: Map[String, Any]): Seq[String] = { param.get(_name) match { diff --git a/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/expr/Expr.scala b/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/expr/Expr.scala index b789a3ccf..5f7d51232 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/expr/Expr.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/expr/Expr.scala @@ -22,4 +22,6 @@ trait Expr extends TreeNode with Serializable { def desc: String + def coalesceDesc: String + } diff --git a/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/expr/FunctionExpr.scala b/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/expr/FunctionExpr.scala index 408b46627..272291e60 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/expr/FunctionExpr.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/expr/FunctionExpr.scala @@ -22,7 +22,6 @@ case class FunctionExpr(functionName: String, args: Seq[Expr]) extends Expr { addChildren(args) - def desc: String = { - s"${functionName}(${args.map(_.desc).mkString(", ")})" - } + def desc: String = s"${functionName}(${args.map(_.desc).mkString(", ")})" + def coalesceDesc: String = desc } \ No newline at end of file diff --git a/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/expr/LiteralExpr.scala b/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/expr/LiteralExpr.scala index 375b5f52d..9b0cc29a3 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/expr/LiteralExpr.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/expr/LiteralExpr.scala @@ -21,6 +21,7 @@ package org.apache.griffin.measure.rules.dsl.expr import org.apache.griffin.measure.utils.TimeUtil trait LiteralExpr extends Expr { + def coalesceDesc: String = desc } case class LiteralNullExpr(str: String) extends LiteralExpr { diff --git a/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/expr/LogicalExpr.scala b/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/expr/LogicalExpr.scala index 6aada8361..1df88c96c 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/expr/LogicalExpr.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/expr/LogicalExpr.scala @@ -29,6 +29,10 @@ case class InExpr(head: Expr, is: Boolean, range: Seq[Expr]) extends LogicalExpr val notStr = if (is) "" else " NOT" s"${head.desc}${notStr} IN (${range.map(_.desc).mkString(", ")})" } + def coalesceDesc: String = { + val notStr = if (is) "" else " NOT" + s"${head.coalesceDesc}${notStr} IN (${range.map(_.coalesceDesc).mkString(", ")})" + } } case class BetweenExpr(head: Expr, is: Boolean, range: Seq[Expr]) extends LogicalExpr { @@ -46,6 +50,14 @@ case class BetweenExpr(head: Expr, is: Boolean, range: Seq[Expr]) extends Logica } s"${head.desc}${notStr} BETWEEN ${rangeStr}" } + def coalesceDesc: String = { + val notStr = if (is) "" else " NOT" + val rangeStr = range match { + case first :: second :: _ => s"${first.coalesceDesc} AND ${second.coalesceDesc}" + case _ => throw new Exception("between expression exception: range less than 2") + } + s"${head.coalesceDesc}${notStr} BETWEEN ${rangeStr}" + } } case class LikeExpr(head: Expr, is: Boolean, value: Expr) extends LogicalExpr { @@ -56,6 +68,10 @@ case class LikeExpr(head: Expr, is: Boolean, value: Expr) extends LogicalExpr { val notStr = if (is) "" else " NOT" s"${head.desc}${notStr} LIKE ${value.desc}" } + def coalesceDesc: String = { + val notStr = if (is) "" else " NOT" + s"${head.coalesceDesc}${notStr} LIKE ${value.coalesceDesc}" + } } case class IsNullExpr(head: Expr, is: Boolean) extends LogicalExpr { @@ -66,6 +82,7 @@ case class IsNullExpr(head: Expr, is: Boolean) extends LogicalExpr { val notStr = if (is) "" else " NOT" s"${head.desc} IS${notStr} NULL" } + def coalesceDesc: String = desc } case class IsNanExpr(head: Expr, is: Boolean) extends LogicalExpr { @@ -76,6 +93,7 @@ case class IsNanExpr(head: Expr, is: Boolean) extends LogicalExpr { val notStr = if (is) "" else "NOT " s"${notStr}isnan(${head.desc})" } + def coalesceDesc: String = desc } // ----------- @@ -84,9 +102,8 @@ case class LogicalFactorExpr(factor: Expr, withBracket: Boolean) extends Logical addChild(factor) - def desc: String = { - if (withBracket) s"(${factor.desc})" else factor.desc - } + def desc: String = if (withBracket) s"(${factor.desc})" else factor.desc + def coalesceDesc: String = factor.coalesceDesc } case class UnaryLogicalExpr(oprs: Seq[String], factor: LogicalExpr) extends LogicalExpr { @@ -98,6 +115,11 @@ case class UnaryLogicalExpr(oprs: Seq[String], factor: LogicalExpr) extends Logi s"(${trans(opr)} ${fac})" } } + def coalesceDesc: String = { + oprs.foldRight(factor.coalesceDesc) { (opr, fac) => + s"(${trans(opr)} ${fac})" + } + } private def trans(s: String): String = { s match { case "!" => "NOT" @@ -117,6 +139,13 @@ case class BinaryLogicalExpr(factor: LogicalExpr, tails: Seq[(String, LogicalExp } if (tails.size <= 0) res else s"${res}" } + def coalesceDesc: String = { + val res = tails.foldLeft(factor.coalesceDesc) { (fac, tail) => + val (opr, expr) = tail + s"${fac} ${trans(opr)} ${expr.coalesceDesc}" + } + if (tails.size <= 0) res else s"${res}" + } private def trans(s: String): String = { s match { case "&&" => "AND" diff --git a/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/expr/MathExpr.scala b/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/expr/MathExpr.scala index 3af54b482..668d2cef7 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/expr/MathExpr.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/expr/MathExpr.scala @@ -25,9 +25,8 @@ case class MathFactorExpr(factor: Expr, withBracket: Boolean) extends MathExpr { addChild(factor) - def desc: String = { - if (withBracket) s"(${factor.desc})" else factor.desc - } + def desc: String = if (withBracket) s"(${factor.desc})" else factor.desc + def coalesceDesc: String = factor.coalesceDesc } case class UnaryMathExpr(oprs: Seq[String], factor: MathExpr) extends MathExpr { @@ -39,6 +38,11 @@ case class UnaryMathExpr(oprs: Seq[String], factor: MathExpr) extends MathExpr { s"(${opr}${fac})" } } + def coalesceDesc: String = { + oprs.foldRight(factor.coalesceDesc) { (opr, fac) => + s"(${opr}${fac})" + } + } } case class BinaryMathExpr(factor: MathExpr, tails: Seq[(String, MathExpr)]) extends MathExpr { @@ -52,4 +56,11 @@ case class BinaryMathExpr(factor: MathExpr, tails: Seq[(String, MathExpr)]) exte } if (tails.size <= 0) res else s"${res}" } + def coalesceDesc: String = { + val res = tails.foldLeft(factor.coalesceDesc) { (fac, tail) => + val (opr, expr) = tail + s"${fac} ${opr} ${expr.coalesceDesc}" + } + if (tails.size <= 0) res else s"${res}" + } } \ No newline at end of file diff --git a/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/expr/SelectExpr.scala b/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/expr/SelectExpr.scala index 7eb571415..1273e0302 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/expr/SelectExpr.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/expr/SelectExpr.scala @@ -24,6 +24,7 @@ trait HeadExpr extends Expr { case class DataSourceHeadExpr(name: String) extends HeadExpr { def desc: String = name + def coalesceDesc: String = desc } case class OtherHeadExpr(expr: Expr) extends HeadExpr { @@ -31,6 +32,7 @@ case class OtherHeadExpr(expr: Expr) extends HeadExpr { addChild(expr) def desc: String = expr.desc + def coalesceDesc: String = expr.coalesceDesc } // ------------- @@ -43,6 +45,7 @@ trait SelectExpr extends Expr { case class FieldSelectExpr(field: String) extends SelectExpr { def desc: String = s".${field}" + def coalesceDesc: String = desc def alias: Option[String] = Some(field) } @@ -51,6 +54,7 @@ case class IndexSelectExpr(index: Expr) extends SelectExpr { addChild(index) def desc: String = s"[${index.desc}]" + def coalesceDesc: String = desc def alias: Option[String] = Some(desc) } @@ -59,6 +63,7 @@ case class FunctionSelectExpr(functionName: String, args: Seq[Expr]) extends Sel addChildren(args) def desc: String = "" + def coalesceDesc: String = desc def alias: Option[String] = None } @@ -79,6 +84,13 @@ case class SelectionExpr(head: HeadExpr, selectors: Seq[SelectExpr], aliasOpt: O } } } + def coalesceDesc: String = { + selectors.lastOption match { + case None => desc + case Some(sel: FunctionSelectExpr) => desc + case _ => s"coalesce(${desc}, 'null')" + } + } def alias: Option[String] = { if (aliasOpt.isEmpty) { selectors.lastOption match { diff --git a/measure/src/main/scala/org/apache/griffin/measure/rules/step/DfOprStep.scala b/measure/src/main/scala/org/apache/griffin/measure/rules/step/DfOprStep.scala index 0581c1781..d04a01175 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rules/step/DfOprStep.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rules/step/DfOprStep.scala @@ -20,8 +20,11 @@ package org.apache.griffin.measure.rules.step import org.apache.griffin.measure.rules.dsl._ -case class DfOprStep(name: String, rule: String, persistType: PersistType) extends ConcreteRuleStep { +case class DfOprStep(name: String, rule: String, details: Map[String, Any] + ) extends ConcreteRuleStep { val dslType: DslType = DfOprType + val persistType: PersistType = NonePersistType + } diff --git a/measure/src/test/resources/config-test1.json b/measure/src/test/resources/config-test1.json index 8459e1f4b..53a876584 100644 --- a/measure/src/test/resources/config-test1.json +++ b/measure/src/test/resources/config-test1.json @@ -34,9 +34,12 @@ "evaluateRule": { "rules": [ { - "dsl.type": "spark-opr", + "dsl.type": "df-opr", "name": "source", - "rule": "from_json(source.value)" + "rule": "from_json", + "details": { + "df.name": "source" + } }, { "dsl.type": "spark-sql", @@ -44,9 +47,13 @@ "rule": "SELECT explode(seeds) as seed FROM source" }, { - "dsl.type": "spark-opr", + "dsl.type": "df-opr", "name": "seeds", - "rule": "from_json(seeds.seed)" + "rule": "from_json", + "details": { + "df.name": "seeds", + "col.name": "seed" + } }, { "dsl.type": "spark-sql", diff --git a/measure/src/test/scala/org/apache/griffin/measure/process/JsonParseTest.scala b/measure/src/test/scala/org/apache/griffin/measure/process/JsonParseTest.scala index 9d24c5424..9657078c4 100644 --- a/measure/src/test/scala/org/apache/griffin/measure/process/JsonParseTest.scala +++ b/measure/src/test/scala/org/apache/griffin/measure/process/JsonParseTest.scala @@ -26,6 +26,7 @@ import org.apache.griffin.measure.config.reader.ParamReaderFactory import org.apache.griffin.measure.config.validator.AllParamValidator import org.apache.griffin.measure.log.Loggable import org.apache.griffin.measure.persist.PersistThreadPool +import org.apache.griffin.measure.process.engine.DataFrameOprs import org.apache.griffin.measure.utils.JsonUtil import org.apache.hadoop.hive.ql.exec.UDF import org.apache.spark.{SparkConf, SparkContext} @@ -81,16 +82,20 @@ class JsonParseTest extends FunSuite with Matchers with BeforeAndAfter with Logg // df2.registerTempTable("df2") + // 1. read from json string to extracted json row - val readSql = "SELECT value FROM src" - val df = sqlContext.sql(readSql) - val rdd = df.map { row => - row.getAs[String]("value") - } - val df1 = sqlContext.read.json(rdd) - df1.printSchema - df1.show(10) - df1.registerTempTable("df1") +// val readSql = "SELECT value FROM src" +// val df = sqlContext.sql(readSql) +// val df = sqlContext.table("src") +// val rdd = df.map { row => +// row.getAs[String]("value") +// } +// val df1 = sqlContext.read.json(rdd) +// df1.printSchema +// df1.show(10) +// df1.registerTempTable("df1") + val details = Map[String, Any](("df.name" -> "src")) + DataFrameOprs.fromJson(sqlContext, "df1", details) // 2. extract json array into lines // val rdd2 = df1.flatMap { row => From 8529fe996c8235a5fe8792f7a135edfb2698f560 Mon Sep 17 00:00:00 2001 From: Lionel Liu Date: Mon, 28 Aug 2017 12:17:35 +0800 Subject: [PATCH 054/111] first step: call method --- .../griffin/measure/persist/HdfsPersist.scala | 2 +- .../process/engine/DataFrameOprEngine.scala | 9 +++- .../process/engine/SparkSqlEngine.scala | 43 ++++++++++++++----- 3 files changed, 40 insertions(+), 14 deletions(-) diff --git a/measure/src/main/scala/org/apache/griffin/measure/persist/HdfsPersist.scala b/measure/src/main/scala/org/apache/griffin/measure/persist/HdfsPersist.scala index 1008e32d4..a4a9ecb8f 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/persist/HdfsPersist.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/persist/HdfsPersist.scala @@ -56,7 +56,7 @@ case class HdfsPersist(config: Map[String, Any], metricName: String, timeStamp: } def available(): Boolean = { - (path.nonEmpty) && (maxPersistLines < Int.MaxValue) + path.nonEmpty } private def persistHead: String = { diff --git a/measure/src/main/scala/org/apache/griffin/measure/process/engine/DataFrameOprEngine.scala b/measure/src/main/scala/org/apache/griffin/measure/process/engine/DataFrameOprEngine.scala index fc454e385..34f4ac0d5 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/process/engine/DataFrameOprEngine.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/process/engine/DataFrameOprEngine.scala @@ -18,6 +18,8 @@ under the License. */ package org.apache.griffin.measure.process.engine +import java.util.Date + import org.apache.griffin.measure.config.params.user.DataSourceParam import org.apache.griffin.measure.data.source.{DataSource, DataSourceFactory} import org.apache.griffin.measure.persist.Persist @@ -58,10 +60,13 @@ case class DataFrameOprEngine(sqlContext: SQLContext, @transient ssc: StreamingC } def persistResult(ruleStep: ConcreteRuleStep, persist: Persist): Boolean = { + val curTime = new Date().getTime ruleStep match { - case SparkSqlStep(_, _, _) => { + case DfOprStep(name, _, _) => { try { - // fixme + val nonLog = s"[ ${name} ] not persisted" + persist.log(curTime, nonLog) + true } catch { case e: Throwable => { diff --git a/measure/src/main/scala/org/apache/griffin/measure/process/engine/SparkSqlEngine.scala b/measure/src/main/scala/org/apache/griffin/measure/process/engine/SparkSqlEngine.scala index 8c24ba6dd..0cedc454e 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/process/engine/SparkSqlEngine.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/process/engine/SparkSqlEngine.scala @@ -18,9 +18,12 @@ under the License. */ package org.apache.griffin.measure.process.engine +import java.util.Date + import org.apache.griffin.measure.config.params.user.DataSourceParam import org.apache.griffin.measure.data.source._ import org.apache.griffin.measure.persist.Persist +import org.apache.griffin.measure.rules.dsl.{MetricPersistType, RecordPersistType} import org.apache.griffin.measure.rules.step._ import org.apache.griffin.measure.utils.JsonUtil import org.apache.spark.sql.{DataFrame, SQLContext} @@ -52,18 +55,40 @@ case class SparkSqlEngine(sqlContext: SQLContext, @transient ssc: StreamingConte } def persistResult(ruleStep: ConcreteRuleStep, persist: Persist): Boolean = { + val curTime = new Date().getTime ruleStep match { - case SparkSqlStep(_, _, _) => { + case SparkSqlStep(name, _, persistType) => { try { - val pdf = sqlContext.sql(getDfSql(ruleStep.name)) - val records = pdf.toJSON.collect() - val persistType = ruleStep.persistType - // fixme - records.foreach(println) + persistType match { + case RecordPersistType => { + val pdf = sqlContext.table(name) + val recordRdd = pdf.toJSON + + persist.records(recordRdd, name) + + val recordLog = s"[ ${name} ] persist records" + persist.log(curTime, recordLog) + } + case MetricPersistType => { + val pdf = sqlContext.table(name) + val recordRdd = pdf.toJSON + + val metric = recordRdd.collect + persist.records(metric, name) + + val metricLog = s"[ ${name} ] persist metric \n${metric.mkString("\n")}" + persist.log(curTime, metricLog) + } + case _ => { + val nonLog = s"[ ${name} ] not persisted" + persist.log(curTime, nonLog) + } + } + true } catch { case e: Throwable => { - error(s"persist result ${ruleStep.name} error: ${e.getMessage}") + error(s"persist result ${name} error: ${e.getMessage}") false } } @@ -72,10 +97,6 @@ case class SparkSqlEngine(sqlContext: SQLContext, @transient ssc: StreamingConte } } - private def getDfSql(name: String): String = { - s"SELECT * FROM `${name}`" - } - } From 3f88e434d7df6cbb0729f975f55c92de48f0404b Mon Sep 17 00:00:00 2001 From: Lionel Liu Date: Mon, 28 Aug 2017 15:11:13 +0800 Subject: [PATCH 055/111] v1 --- .../griffin/measure/persist/HdfsPersist.scala | 165 +++++++++++------ .../griffin/measure/persist/HttpPersist.scala | 49 +++-- .../measure/persist/LoggerPersist.scala | 131 +++++++------ .../measure/persist/MultiPersists.scala | 11 +- .../measure/persist/OldHttpPersist.scala | 174 +++++++++--------- .../griffin/measure/persist/Persist.scala | 21 ++- .../measure/persist/PersistFactory.scala | 4 +- .../process/engine/DataFrameOprEngine.scala | 8 +- .../process/engine/SparkSqlEngine.scala | 14 +- .../rules/adaptor/GriffinDslAdaptor.scala | 74 +++++--- .../griffin/measure/utils/HdfsUtil.scala | 2 +- measure/src/test/resources/config-test.json | 15 +- measure/src/test/resources/config-test2.json | 17 +- measure/src/test/resources/env-test.json | 8 + .../measure/process/JsonParseTest.scala | 3 +- 15 files changed, 409 insertions(+), 287 deletions(-) diff --git a/measure/src/main/scala/org/apache/griffin/measure/persist/HdfsPersist.scala b/measure/src/main/scala/org/apache/griffin/measure/persist/HdfsPersist.scala index a4a9ecb8f..63835620b 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/persist/HdfsPersist.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/persist/HdfsPersist.scala @@ -92,30 +92,112 @@ case class HdfsPersist(config: Map[String, Any], metricName: String, timeStamp: } } - def result(rt: Long, result: Result): Unit = { - try { - val resStr = result match { - case ar: AccuracyResult => { - s"match percentage: ${ar.matchPercentage}\ntotal count: ${ar.getTotal}\nmiss count: ${ar.getMiss}, match count: ${ar.getMatch}" - } - case pr: ProfileResult => { - s"match percentage: ${pr.matchPercentage}\ntotal count: ${pr.getTotal}\nmiss count: ${pr.getMiss}, match count: ${pr.getMatch}" - } - case _ => { - s"result: ${result}" - } - } - HdfsUtil.writeContent(ResultFile, timeHead(rt) + resStr) - log(rt, resStr) +// def result(rt: Long, result: Result): Unit = { +// try { +// val resStr = result match { +// case ar: AccuracyResult => { +// s"match percentage: ${ar.matchPercentage}\ntotal count: ${ar.getTotal}\nmiss count: ${ar.getMiss}, match count: ${ar.getMatch}" +// } +// case pr: ProfileResult => { +// s"match percentage: ${pr.matchPercentage}\ntotal count: ${pr.getTotal}\nmiss count: ${pr.getMiss}, match count: ${pr.getMatch}" +// } +// case _ => { +// s"result: ${result}" +// } +// } +// HdfsUtil.writeContent(ResultFile, timeHead(rt) + resStr) +// log(rt, resStr) +// +// info(resStr) +// } catch { +// case e: Throwable => error(e.getMessage) +// } +// } + + // need to avoid string too long +// private def rddRecords(records: RDD[String], path: String): Unit = { +// try { +// val recordCount = records.count +// val count = if (maxPersistLines < 0) recordCount else scala.math.min(maxPersistLines, recordCount) +// if (count > 0) { +// val groupCount = ((count - 1) / maxLinesPerFile + 1).toInt +// if (groupCount <= 1) { +// val recs = records.take(count.toInt) +// persistRecords(path, recs) +// } else { +// val groupedRecords: RDD[(Long, Iterable[String])] = +// records.zipWithIndex.flatMap { r => +// val gid = r._2 / maxLinesPerFile +// if (gid < groupCount) Some((gid, r._1)) else None +// }.groupByKey() +// groupedRecords.foreach { group => +// val (gid, recs) = group +// val hdfsPath = if (gid == 0) path else withSuffix(path, gid.toString) +// persistRecords(hdfsPath, recs) +// } +// } +// } +// } catch { +// case e: Throwable => error(e.getMessage) +// } +// } +// +// private def iterableRecords(records: Iterable[String], path: String): Unit = { +// try { +// val recordCount = records.size +// val count = if (maxPersistLines < 0) recordCount else scala.math.min(maxPersistLines, recordCount) +// if (count > 0) { +// val groupCount = ((count - 1) / maxLinesPerFile + 1).toInt +// if (groupCount <= 1) { +// val recs = records.take(count.toInt) +// persistRecords(path, recs) +// } else { +// val groupedRecords = records.grouped(groupCount).zipWithIndex +// groupedRecords.take(groupCount).foreach { group => +// val (recs, gid) = group +// val hdfsPath = if (gid == 0) path else withSuffix(path, gid.toString) +// persistRecords(hdfsPath, recs) +// } +// } +// } +// } catch { +// case e: Throwable => error(e.getMessage) +// } +// } +// +// def records(recs: RDD[String], tp: String): Unit = { +// tp match { +// case PersistDataType.MISS => rddRecords(recs, MissRecFile) +// case PersistDataType.MATCH => rddRecords(recs, MatchRecFile) +// case _ => {} +// } +// } +// +// def records(recs: Iterable[String], tp: String): Unit = { +// tp match { +// case PersistDataType.MISS => iterableRecords(recs, MissRecFile) +// case PersistDataType.MATCH => iterableRecords(recs, MatchRecFile) +// case _ => {} +// } +// } - info(resStr) + private def persistRecords(hdfsPath: String, records: Iterable[String]): Unit = { + val recStr = records.mkString("\n") + HdfsUtil.writeContent(hdfsPath, recStr) + } + + def log(rt: Long, msg: String): Unit = { + try { + val logStr = (if (isInit) persistHead else "") + timeHead(rt) + s"${msg}\n\n" + HdfsUtil.appendContent(LogFile, logStr) } catch { case e: Throwable => error(e.getMessage) } } - // need to avoid string too long - private def rddRecords(records: RDD[String], path: String): Unit = { + + def persistRecords(records: RDD[String], name: String): Unit = { + val path = filePath(name) try { val recordCount = records.count val count = if (maxPersistLines < 0) recordCount else scala.math.min(maxPersistLines, recordCount) @@ -142,17 +224,18 @@ case class HdfsPersist(config: Map[String, Any], metricName: String, timeStamp: } } - private def iterableRecords(records: Iterable[String], path: String): Unit = { + def persistMetrics(metrics: Seq[String], name: String): Unit = { + val path = filePath(name) try { - val recordCount = records.size + val recordCount = metrics.size val count = if (maxPersistLines < 0) recordCount else scala.math.min(maxPersistLines, recordCount) if (count > 0) { val groupCount = ((count - 1) / maxLinesPerFile + 1).toInt if (groupCount <= 1) { - val recs = records.take(count.toInt) + val recs = metrics.take(count.toInt) persistRecords(path, recs) } else { - val groupedRecords = records.grouped(groupCount).zipWithIndex + val groupedRecords = metrics.grouped(groupCount).zipWithIndex groupedRecords.take(groupCount).foreach { group => val (recs, gid) = group val hdfsPath = if (gid == 0) path else withSuffix(path, gid.toString) @@ -165,42 +248,4 @@ case class HdfsPersist(config: Map[String, Any], metricName: String, timeStamp: } } - def records(recs: RDD[String], tp: String): Unit = { - tp match { - case PersistDataType.MISS => rddRecords(recs, MissRecFile) - case PersistDataType.MATCH => rddRecords(recs, MatchRecFile) - case _ => {} - } - } - - def records(recs: Iterable[String], tp: String): Unit = { - tp match { - case PersistDataType.MISS => iterableRecords(recs, MissRecFile) - case PersistDataType.MATCH => iterableRecords(recs, MatchRecFile) - case _ => {} - } - } - -// def missRecords(records: RDD[String]): Unit = { -// rddRecords(records, MissRecFile) -// } -// -// def matchRecords(records: RDD[String]): Unit = { -// rddRecords(records, MatchRecFile) -// } - - private def persistRecords(hdfsPath: String, records: Iterable[String]): Unit = { - val recStr = records.mkString("\n") - HdfsUtil.writeContent(hdfsPath, recStr) - } - - def log(rt: Long, msg: String): Unit = { - try { - val logStr = (if (isInit) persistHead else "") + timeHead(rt) + s"${msg}\n\n" - HdfsUtil.appendContent(LogFile, logStr) - } catch { - case e: Throwable => error(e.getMessage) - } - } - } diff --git a/measure/src/main/scala/org/apache/griffin/measure/persist/HttpPersist.scala b/measure/src/main/scala/org/apache/griffin/measure/persist/HttpPersist.scala index 6d5bac3e5..528905421 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/persist/HttpPersist.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/persist/HttpPersist.scala @@ -40,21 +40,21 @@ case class HttpPersist(config: Map[String, Any], metricName: String, timeStamp: def start(msg: String): Unit = {} def finish(): Unit = {} - def result(rt: Long, result: Result): Unit = { - result match { - case ar: AccuracyResult => { - val dataMap = Map[String, Any](("name" -> metricName), ("tmst" -> timeStamp), ("total" -> ar.getTotal), ("matched" -> ar.getMatch)) - httpResult(dataMap) - } - case pr: ProfileResult => { - val dataMap = Map[String, Any](("name" -> metricName), ("tmst" -> timeStamp), ("total" -> pr.getTotal), ("matched" -> pr.getMatch)) - httpResult(dataMap) - } - case _ => { - info(s"result: ${result}") - } - } - } +// def result(rt: Long, result: Result): Unit = { +// result match { +// case ar: AccuracyResult => { +// val dataMap = Map[String, Any](("name" -> metricName), ("tmst" -> timeStamp), ("total" -> ar.getTotal), ("matched" -> ar.getMatch)) +// httpResult(dataMap) +// } +// case pr: ProfileResult => { +// val dataMap = Map[String, Any](("name" -> metricName), ("tmst" -> timeStamp), ("total" -> pr.getTotal), ("matched" -> pr.getMatch)) +// httpResult(dataMap) +// } +// case _ => { +// info(s"result: ${result}") +// } +// } +// } private def httpResult(dataMap: Map[String, Any]) = { try { @@ -77,12 +77,27 @@ case class HttpPersist(config: Map[String, Any], metricName: String, timeStamp: } - def records(recs: RDD[String], tp: String): Unit = {} - def records(recs: Iterable[String], tp: String): Unit = {} +// def records(recs: RDD[String], tp: String): Unit = {} +// def records(recs: Iterable[String], tp: String): Unit = {} // def missRecords(records: RDD[String]): Unit = {} // def matchRecords(records: RDD[String]): Unit = {} def log(rt: Long, msg: String): Unit = {} + def persistRecords(records: RDD[String], name: String): Unit = {} + + def persistMetrics(metrics: Seq[String], name: String): Unit = { + val maps = metrics.flatMap { m => + try { + Some(JsonUtil.toAnyMap(m) ++ Map[String, Any](("name" -> metricName), ("tmst" -> timeStamp))) + } catch { + case e: Throwable => None + } + } + maps.foreach { map => + httpResult(map) + } + } + } diff --git a/measure/src/main/scala/org/apache/griffin/measure/persist/LoggerPersist.scala b/measure/src/main/scala/org/apache/griffin/measure/persist/LoggerPersist.scala index 753aa80ea..0bcba6b75 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/persist/LoggerPersist.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/persist/LoggerPersist.scala @@ -40,79 +40,106 @@ case class LoggerPersist(config: Map[String, Any], metricName: String, timeStamp println(s"[${timeStamp}] ${metricName} finish") } - def result(rt: Long, result: Result): Unit = { - try { - val resStr = result match { - case ar: AccuracyResult => { - s"match percentage: ${ar.matchPercentage}\ntotal count: ${ar.getTotal}\nmiss count: ${ar.getMiss}, match count: ${ar.getMatch}" - } - case pr: ProfileResult => { - s"match percentage: ${pr.matchPercentage}\ntotal count: ${pr.getTotal}\nmiss count: ${pr.getMiss}, match count: ${pr.getMatch}" - } - case _ => { - s"result: ${result}" - } - } - println(s"[${timeStamp}] ${metricName} result: \n${resStr}") - } catch { - case e: Throwable => error(e.getMessage) - } +// def result(rt: Long, result: Result): Unit = { +// try { +// val resStr = result match { +// case ar: AccuracyResult => { +// s"match percentage: ${ar.matchPercentage}\ntotal count: ${ar.getTotal}\nmiss count: ${ar.getMiss}, match count: ${ar.getMatch}" +// } +// case pr: ProfileResult => { +// s"match percentage: ${pr.matchPercentage}\ntotal count: ${pr.getTotal}\nmiss count: ${pr.getMiss}, match count: ${pr.getMatch}" +// } +// case _ => { +// s"result: ${result}" +// } +// } +// println(s"[${timeStamp}] ${metricName} result: \n${resStr}") +// } catch { +// case e: Throwable => error(e.getMessage) +// } +// } +// +// // need to avoid string too long +// private def rddRecords(records: RDD[String]): Unit = { +// try { +// val recordCount = records.count.toInt +// val count = if (maxLogLines < 0) recordCount else scala.math.min(maxLogLines, recordCount) +// if (count > 0) { +// val recordsArray = records.take(count) +//// recordsArray.foreach(println) +// } +// } catch { +// case e: Throwable => error(e.getMessage) +// } +// } + +// private def iterableRecords(records: Iterable[String]): Unit = { +// try { +// val recordCount = records.size +// val count = if (maxLogLines < 0) recordCount else scala.math.min(maxLogLines, recordCount) +// if (count > 0) { +// val recordsArray = records.take(count) +//// recordsArray.foreach(println) +// } +// } catch { +// case e: Throwable => error(e.getMessage) +// } +// } + +// def records(recs: RDD[String], tp: String): Unit = { +// tp match { +// case PersistDataType.MISS => rddRecords(recs) +// case PersistDataType.MATCH => rddRecords(recs) +// case _ => {} +// } +// } +// +// def records(recs: Iterable[String], tp: String): Unit = { +// tp match { +// case PersistDataType.MISS => iterableRecords(recs) +// case PersistDataType.MATCH => iterableRecords(recs) +// case _ => {} +// } +// } + +// def missRecords(records: RDD[String]): Unit = { +// warn(s"[${timeStamp}] ${metricName} miss records: ") +// rddRecords(records) +// } +// def matchRecords(records: RDD[String]): Unit = { +// warn(s"[${timeStamp}] ${metricName} match records: ") +// rddRecords(records) +// } + + def log(rt: Long, msg: String): Unit = { + println(s"[${timeStamp}] ${rt}: ${msg}") } - // need to avoid string too long - private def rddRecords(records: RDD[String]): Unit = { + def persistRecords(records: RDD[String], name: String): Unit = { try { val recordCount = records.count.toInt val count = if (maxLogLines < 0) recordCount else scala.math.min(maxLogLines, recordCount) if (count > 0) { val recordsArray = records.take(count) -// recordsArray.foreach(println) + recordsArray.foreach(println) } } catch { case e: Throwable => error(e.getMessage) } } - private def iterableRecords(records: Iterable[String]): Unit = { + def persistMetrics(metrics: Seq[String], name: String): Unit = { try { - val recordCount = records.size + val recordCount = metrics.size val count = if (maxLogLines < 0) recordCount else scala.math.min(maxLogLines, recordCount) if (count > 0) { - val recordsArray = records.take(count) -// recordsArray.foreach(println) + val recordsArray = metrics.take(count) + recordsArray.foreach(println) } } catch { case e: Throwable => error(e.getMessage) } } - def records(recs: RDD[String], tp: String): Unit = { - tp match { - case PersistDataType.MISS => rddRecords(recs) - case PersistDataType.MATCH => rddRecords(recs) - case _ => {} - } - } - - def records(recs: Iterable[String], tp: String): Unit = { - tp match { - case PersistDataType.MISS => iterableRecords(recs) - case PersistDataType.MATCH => iterableRecords(recs) - case _ => {} - } - } - -// def missRecords(records: RDD[String]): Unit = { -// warn(s"[${timeStamp}] ${metricName} miss records: ") -// rddRecords(records) -// } -// def matchRecords(records: RDD[String]): Unit = { -// warn(s"[${timeStamp}] ${metricName} match records: ") -// rddRecords(records) -// } - - def log(rt: Long, msg: String): Unit = { - println(s"[${timeStamp}] ${rt}: ${msg}") - } } diff --git a/measure/src/main/scala/org/apache/griffin/measure/persist/MultiPersists.scala b/measure/src/main/scala/org/apache/griffin/measure/persist/MultiPersists.scala index 25c8b0bb7..095147e27 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/persist/MultiPersists.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/persist/MultiPersists.scala @@ -39,14 +39,17 @@ case class MultiPersists(persists: Iterable[Persist]) extends Persist { def start(msg: String): Unit = { persists.foreach(_.start(msg)) } def finish(): Unit = { persists.foreach(_.finish()) } - def result(rt: Long, result: Result): Unit = { persists.foreach(_.result(rt, result)) } - - def records(recs: RDD[String], tp: String): Unit = { persists.foreach(_.records(recs, tp)) } - def records(recs: Iterable[String], tp: String): Unit = { persists.foreach(_.records(recs, tp)) } +// def result(rt: Long, result: Result): Unit = { persists.foreach(_.result(rt, result)) } +// +// def records(recs: RDD[String], tp: String): Unit = { persists.foreach(_.records(recs, tp)) } +// def records(recs: Iterable[String], tp: String): Unit = { persists.foreach(_.records(recs, tp)) } // def missRecords(records: RDD[String]): Unit = { persists.foreach(_.missRecords(records)) } // def matchRecords(records: RDD[String]): Unit = { persists.foreach(_.matchRecords(records)) } def log(rt: Long, msg: String): Unit = { persists.foreach(_.log(rt, msg)) } + def persistRecords(records: RDD[String], name: String): Unit = { persists.foreach(_.persistRecords(records, name)) } + def persistMetrics(metrics: Seq[String], name: String): Unit = { persists.foreach(_.persistMetrics(metrics, name)) } + } diff --git a/measure/src/main/scala/org/apache/griffin/measure/persist/OldHttpPersist.scala b/measure/src/main/scala/org/apache/griffin/measure/persist/OldHttpPersist.scala index 357d6e146..84316b365 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/persist/OldHttpPersist.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/persist/OldHttpPersist.scala @@ -1,87 +1,87 @@ -/* -Licensed to the Apache Software Foundation (ASF) under one -or more contributor license agreements. See the NOTICE file -distributed with this work for additional information -regarding copyright ownership. The ASF licenses this file -to you under the Apache License, Version 2.0 (the -"License"); you may not use this file except in compliance -with the License. You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, -software distributed under the License is distributed on an -"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -KIND, either express or implied. See the License for the -specific language governing permissions and limitations -under the License. -*/ -package org.apache.griffin.measure.persist - -import org.apache.griffin.measure.result._ -import org.apache.griffin.measure.utils.{HttpUtil, JsonUtil} -import org.apache.spark.rdd.RDD - -// persist result by old http way -- temporary way -case class OldHttpPersist(config: Map[String, Any], metricName: String, timeStamp: Long) extends Persist { - - val Api = "api" - val Method = "method" - - val api = config.getOrElse(Api, "").toString - val method = config.getOrElse(Method, "post").toString - - def available(): Boolean = { - api.nonEmpty - } - - def start(msg: String): Unit = {} - def finish(): Unit = {} - - def result(rt: Long, result: Result): Unit = { - result match { - case ar: AccuracyResult => { - val matchPercentage: Double = if (ar.getTotal <= 0) 0 else (ar.getMatch * 1.0 / ar.getTotal) * 100 - val dataMap = Map[String, Any](("metricName" -> metricName), ("timestamp" -> timeStamp), ("value" -> matchPercentage), ("count" -> ar.getTotal)) - httpResult(dataMap) - } - case pr: ProfileResult => { - val dataMap = Map[String, Any](("metricName" -> metricName), ("timestamp" -> timeStamp), ("value" -> pr.getMatch), ("count" -> pr.getTotal)) - httpResult(dataMap) - } - case _ => { - info(s"result: ${result}") - } - } - } - - private def httpResult(dataMap: Map[String, Any]) = { - try { - val data = JsonUtil.toJson(dataMap) - // post - val params = Map[String, Object]() - val header = Map[String, Object](("content-type" -> "application/json")) - - def func(): Boolean = { - HttpUtil.httpRequest(api, method, params, header, data) - } - - PersistThreadPool.addTask(func _, 10) - -// val status = HttpUtil.httpRequest(api, method, params, header, data) -// info(s"${method} to ${api} response status: ${status}") - } catch { - case e: Throwable => error(e.getMessage) - } - - } - - def records(recs: RDD[String], tp: String): Unit = {} - def records(recs: Iterable[String], tp: String): Unit = {} - -// def missRecords(records: RDD[String]): Unit = {} -// def matchRecords(records: RDD[String]): Unit = {} - - def log(rt: Long, msg: String): Unit = {} - -} +///* +//Licensed to the Apache Software Foundation (ASF) under one +//or more contributor license agreements. See the NOTICE file +//distributed with this work for additional information +//regarding copyright ownership. The ASF licenses this file +//to you under the Apache License, Version 2.0 (the +//"License"); you may not use this file except in compliance +//with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +//Unless required by applicable law or agreed to in writing, +//software distributed under the License is distributed on an +//"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +//KIND, either express or implied. See the License for the +//specific language governing permissions and limitations +//under the License. +//*/ +//package org.apache.griffin.measure.persist +// +//import org.apache.griffin.measure.result._ +//import org.apache.griffin.measure.utils.{HttpUtil, JsonUtil} +//import org.apache.spark.rdd.RDD +// +//// persist result by old http way -- temporary way +//case class OldHttpPersist(config: Map[String, Any], metricName: String, timeStamp: Long) extends Persist { +// +// val Api = "api" +// val Method = "method" +// +// val api = config.getOrElse(Api, "").toString +// val method = config.getOrElse(Method, "post").toString +// +// def available(): Boolean = { +// api.nonEmpty +// } +// +// def start(msg: String): Unit = {} +// def finish(): Unit = {} +// +// def result(rt: Long, result: Result): Unit = { +// result match { +// case ar: AccuracyResult => { +// val matchPercentage: Double = if (ar.getTotal <= 0) 0 else (ar.getMatch * 1.0 / ar.getTotal) * 100 +// val dataMap = Map[String, Any](("metricName" -> metricName), ("timestamp" -> timeStamp), ("value" -> matchPercentage), ("count" -> ar.getTotal)) +// httpResult(dataMap) +// } +// case pr: ProfileResult => { +// val dataMap = Map[String, Any](("metricName" -> metricName), ("timestamp" -> timeStamp), ("value" -> pr.getMatch), ("count" -> pr.getTotal)) +// httpResult(dataMap) +// } +// case _ => { +// info(s"result: ${result}") +// } +// } +// } +// +// private def httpResult(dataMap: Map[String, Any]) = { +// try { +// val data = JsonUtil.toJson(dataMap) +// // post +// val params = Map[String, Object]() +// val header = Map[String, Object](("content-type" -> "application/json")) +// +// def func(): Boolean = { +// HttpUtil.httpRequest(api, method, params, header, data) +// } +// +// PersistThreadPool.addTask(func _, 10) +// +//// val status = HttpUtil.httpRequest(api, method, params, header, data) +//// info(s"${method} to ${api} response status: ${status}") +// } catch { +// case e: Throwable => error(e.getMessage) +// } +// +// } +// +// def records(recs: RDD[String], tp: String): Unit = {} +// def records(recs: Iterable[String], tp: String): Unit = {} +// +//// def missRecords(records: RDD[String]): Unit = {} +//// def matchRecords(records: RDD[String]): Unit = {} +// +// def log(rt: Long, msg: String): Unit = {} +// +//} diff --git a/measure/src/main/scala/org/apache/griffin/measure/persist/Persist.scala b/measure/src/main/scala/org/apache/griffin/measure/persist/Persist.scala index 7db7f51de..f76e69c5a 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/persist/Persist.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/persist/Persist.scala @@ -35,18 +35,19 @@ trait Persist extends Loggable with Serializable { def start(msg: String): Unit def finish(): Unit - def result(rt: Long, result: Result): Unit + def log(rt: Long, msg: String): Unit - def records(recs: RDD[String], tp: String): Unit - def records(recs: Iterable[String], tp: String): Unit +// def result(rt: Long, result: Result): Unit +// +// def records(recs: RDD[String], tp: String): Unit +// def records(recs: Iterable[String], tp: String): Unit -// def missRecords(records: RDD[String]): Unit -// def matchRecords(records: RDD[String]): Unit + def persistRecords(records: RDD[String], name: String): Unit + def persistMetrics(metrics: Seq[String], name: String): Unit - def log(rt: Long, msg: String): Unit } -object PersistDataType { - final val MISS = "miss" - final val MATCH = "match" -} \ No newline at end of file +//object PersistDataType { +// final val MISS = "miss" +// final val MATCH = "match" +//} \ No newline at end of file diff --git a/measure/src/main/scala/org/apache/griffin/measure/persist/PersistFactory.scala b/measure/src/main/scala/org/apache/griffin/measure/persist/PersistFactory.scala index 43301608c..3a743431e 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/persist/PersistFactory.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/persist/PersistFactory.scala @@ -27,7 +27,7 @@ case class PersistFactory(persistParams: Iterable[PersistParam], metricName: Str val HDFS_REGEX = """^(?i)hdfs$""".r val HTTP_REGEX = """^(?i)http$""".r - val OLDHTTP_REGEX = """^(?i)oldhttp$""".r +// val OLDHTTP_REGEX = """^(?i)oldhttp$""".r val LOG_REGEX = """^(?i)log$""".r def getPersists(timeStamp: Long): MultiPersists = { @@ -40,7 +40,7 @@ case class PersistFactory(persistParams: Iterable[PersistParam], metricName: Str val persistTry = persistParam.persistType match { case HDFS_REGEX() => Try(HdfsPersist(config, metricName, timeStamp)) case HTTP_REGEX() => Try(HttpPersist(config, metricName, timeStamp)) - case OLDHTTP_REGEX() => Try(OldHttpPersist(config, metricName, timeStamp)) +// case OLDHTTP_REGEX() => Try(OldHttpPersist(config, metricName, timeStamp)) case LOG_REGEX() => Try(LoggerPersist(config, metricName, timeStamp)) case _ => throw new Exception("not supported persist type") } diff --git a/measure/src/main/scala/org/apache/griffin/measure/process/engine/DataFrameOprEngine.scala b/measure/src/main/scala/org/apache/griffin/measure/process/engine/DataFrameOprEngine.scala index 34f4ac0d5..6d3f43344 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/process/engine/DataFrameOprEngine.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/process/engine/DataFrameOprEngine.scala @@ -40,7 +40,7 @@ case class DataFrameOprEngine(sqlContext: SQLContext, @transient ssc: StreamingC try { rule match { case DataFrameOprs._fromJson => { - val df = DataFrameOprs.fromJson(sqlContext, name, details) + val df = DataFrameOprs.fromJson(sqlContext, details) df.registerTempTable(name) } case _ => { @@ -85,13 +85,13 @@ object DataFrameOprs { final val _fromJson = "from_json" - def fromJson(sqlContext: SQLContext, name: String, details: Map[String, Any]): DataFrame = { + def fromJson(sqlContext: SQLContext, details: Map[String, Any]): DataFrame = { val _dfName = "df.name" val _colName = "col.name" - val dfName = details.getOrElse(_dfName, name).toString + val dfName = details.getOrElse(_dfName, "").toString val colNameOpt = details.get(_colName).map(_.toString) - val df = sqlContext.table(dfName) + val df = sqlContext.table(s"`${dfName}`") val rdd = colNameOpt match { case Some(colName: String) => df.map(_.getAs[String](colName)) case _ => df.map(_.getAs[String](0)) diff --git a/measure/src/main/scala/org/apache/griffin/measure/process/engine/SparkSqlEngine.scala b/measure/src/main/scala/org/apache/griffin/measure/process/engine/SparkSqlEngine.scala index 0cedc454e..2fe1e8a5f 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/process/engine/SparkSqlEngine.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/process/engine/SparkSqlEngine.scala @@ -61,22 +61,22 @@ case class SparkSqlEngine(sqlContext: SQLContext, @transient ssc: StreamingConte try { persistType match { case RecordPersistType => { - val pdf = sqlContext.table(name) - val recordRdd = pdf.toJSON + val pdf = sqlContext.table(s"`${name}`") + val records = pdf.toJSON - persist.records(recordRdd, name) + persist.persistRecords(records, name) val recordLog = s"[ ${name} ] persist records" persist.log(curTime, recordLog) } case MetricPersistType => { - val pdf = sqlContext.table(name) + val pdf = sqlContext.table(s"`${name}`") val recordRdd = pdf.toJSON - val metric = recordRdd.collect - persist.records(metric, name) + val metrics = recordRdd.collect + persist.persistMetrics(metrics, name) - val metricLog = s"[ ${name} ] persist metric \n${metric.mkString("\n")}" + val metricLog = s"[ ${name} ] persist metric \n${metrics.mkString("\n")}" persist.log(curTime, metricLog) } case _ => { diff --git a/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/GriffinDslAdaptor.scala b/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/GriffinDslAdaptor.scala index 05755a4e4..fa1a5bcf6 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/GriffinDslAdaptor.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/GriffinDslAdaptor.scala @@ -36,9 +36,11 @@ case class GriffinDslAdaptor(dataSourceNames: Seq[String], functionNames: Seq[St object AccuracyInfo { val _Source = "source" val _Target = "target" - val _MissRecord = "miss.record" - val _MissCount = "miss.count" - val _TotalCount = "total.count" + val _MissRecords = "miss.records" + val _Accuracy = "accuracy" + val _Miss = "miss" + val _Total = "total" + val _Matched = "matched" def getNameOpt(param: Map[String, Any], key: String): Option[String] = param.get(key).flatMap(a => Some(a.toString)) def resultName(param: Map[String, Any], key: String): String = { val nameOpt = param.get(key) match { @@ -75,9 +77,8 @@ case class GriffinDslAdaptor(dataSourceNames: Seq[String], functionNames: Seq[St dqType match { case AccuracyType => { Seq[String]( - AccuracyInfo.resultName(param, AccuracyInfo._MissRecord), - AccuracyInfo.resultName(param, AccuracyInfo._MissCount), - AccuracyInfo.resultName(param, AccuracyInfo._TotalCount) + AccuracyInfo.resultName(param, AccuracyInfo._MissRecords), + AccuracyInfo.resultName(param, AccuracyInfo._Accuracy) ) } case ProfilingType => { @@ -136,7 +137,7 @@ case class GriffinDslAdaptor(dataSourceNames: Seq[String], functionNames: Seq[St val analyzer = AccuracyAnalyzer(expr, sourceName, targetName) // 1. miss record - val missRecordSql = { + val missRecordsSql = { val selClause = analyzer.sourceSelectionExprs.map { sel => val alias = sel.alias match { case Some(a) => s" AS ${a}" @@ -157,36 +158,55 @@ case class GriffinDslAdaptor(dataSourceNames: Seq[String], functionNames: Seq[St s"SELECT ${selClause} FROM ${sourceName} LEFT JOIN ${targetName} ON ${onClause} WHERE ${whereClause}" } - val missRecordName = AccuracyInfo.resultName(details, AccuracyInfo._MissRecord) - val missRecordStep = SparkSqlStep( - missRecordName, - missRecordSql, - AccuracyInfo.resultPersistType(details, AccuracyInfo._MissRecord, RecordPersistType) + val missRecordsName = AccuracyInfo.resultName(details, AccuracyInfo._MissRecords) + val missRecordsStep = SparkSqlStep( + missRecordsName, + missRecordsSql, + AccuracyInfo.resultPersistType(details, AccuracyInfo._MissRecords, RecordPersistType) ) // 2. miss count - val missCountSql = { - s"SELECT COUNT(*) AS `miss.count` FROM `${missRecordName}`" + val missTableName = "_miss_" + val missColName = AccuracyInfo.getNameOpt(details, AccuracyInfo._Miss).getOrElse(AccuracyInfo._Miss) + val missSql = { + s"SELECT COUNT(*) AS `${missColName}` FROM `${missRecordsName}`" } - val missCountName = AccuracyInfo.resultName(details, AccuracyInfo._MissCount) - val missCountStep = SparkSqlStep( - missCountName, - missCountSql, - AccuracyInfo.resultPersistType(details, AccuracyInfo._MissCount, MetricPersistType) + val missStep = SparkSqlStep( + missTableName, + missSql, + NonePersistType ) // 3. total count - val totalCountSql = { - s"SELECT COUNT(*) AS `total.count` FROM ${sourceName}" + val totalTableName = "_total_" + val totalColName = AccuracyInfo.getNameOpt(details, AccuracyInfo._Total).getOrElse(AccuracyInfo._Total) + val totalSql = { + s"SELECT COUNT(*) AS `${totalColName}` FROM `${sourceName}`" } - val totalCountName = AccuracyInfo.resultName(details, AccuracyInfo._TotalCount) - val totalCountStep = SparkSqlStep( - totalCountName, - totalCountSql, - AccuracyInfo.resultPersistType(details, AccuracyInfo._TotalCount, MetricPersistType) + val totalStep = SparkSqlStep( + totalTableName, + totalSql, + NonePersistType ) - missRecordStep :: missCountStep :: totalCountStep :: Nil + // 4. accuracy metric + val matchedColName = AccuracyInfo.getNameOpt(details, AccuracyInfo._Matched).getOrElse(AccuracyInfo._Matched) + val accuracyMetricSql = { + s""" + |SELECT `${totalTableName}`.`${totalColName}` AS `${totalColName}`, + |`${missTableName}`.`${missColName}` AS `${missColName}`, + |(`${totalColName}` - `${missColName}`) AS `${matchedColName}` + |FROM `${totalTableName}` JOIN `${missTableName}` + """.stripMargin + } + val accuracyMetricName = AccuracyInfo.resultName(details, AccuracyInfo._Accuracy) + val accuracyMetricStep = SparkSqlStep( + accuracyMetricName, + accuracyMetricSql, + AccuracyInfo.resultPersistType(details, AccuracyInfo._Accuracy, MetricPersistType) + ) + + missRecordsStep :: missStep :: totalStep :: accuracyMetricStep :: Nil } case ProfilingType => { Nil diff --git a/measure/src/main/scala/org/apache/griffin/measure/utils/HdfsUtil.scala b/measure/src/main/scala/org/apache/griffin/measure/utils/HdfsUtil.scala index 6dd54b7f6..079359275 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/utils/HdfsUtil.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/utils/HdfsUtil.scala @@ -27,7 +27,7 @@ object HdfsUtil { private val conf = new Configuration() conf.set("dfs.support.append", "true") -// conf.set("fs.defaultFS", "hdfs://localhost") // debug @localhost + conf.set("fs.defaultFS", "hdfs://localhost") // debug @localhost private val dfs = FileSystem.get(conf) diff --git a/measure/src/test/resources/config-test.json b/measure/src/test/resources/config-test.json index 8172038b6..23eb5ff8f 100644 --- a/measure/src/test/resources/config-test.json +++ b/measure/src/test/resources/config-test.json @@ -33,18 +33,21 @@ "rules": [ { "dsl.type": "spark-sql", - "name": "miss.record", + "name": "miss.records", "rule": "SELECT source.user_id, source.first_name, source.last_name, source.address, source.email, source.phone, source.post_code FROM source LEFT JOIN target ON coalesce(source.user_id, 'null') = coalesce(target.user_id, 'null') AND coalesce(source.first_name, 'null') = coalesce(target.first_name, 'null') AND coalesce(source.last_name, 'null') = coalesce(target.last_name, 'null') AND coalesce(source.address, 'null') = coalesce(target.address, 'null') AND coalesce(source.email, 'null') = coalesce(target.email, 'null') AND coalesce(source.phone, 'null') = coalesce(target.phone, 'null') AND coalesce(source.post_code, 'null') = coalesce(target.post_code, 'null') WHERE (NOT (source.user_id IS NULL AND source.first_name IS NULL AND source.last_name IS NULL AND source.address IS NULL AND source.email IS NULL AND source.phone IS NULL AND source.post_code IS NULL)) AND (target.user_id IS NULL AND target.first_name IS NULL AND target.last_name IS NULL AND target.address IS NULL AND target.email IS NULL AND target.phone IS NULL AND target.post_code IS NULL)", "persist.type": "record" }, { "dsl.type": "spark-sql", - "name": "miss.count", - "rule": "SELECT COUNT(*) AS `miss.count` FROM `miss.record`", - "persist.type": "metric" + "name": "miss", + "rule": "SELECT COUNT(*) AS `miss` FROM `miss.records`", + }, { + "dsl.type": "spark-sql", + "name": "total", + "rule": "SELECT COUNT(*) AS `total` FROM source", }, { "dsl.type": "spark-sql", - "name": "total.count", - "rule": "SELECT COUNT(*) AS `total.count` FROM source", + "name": "accuracy", + "rule": "SELECT `total`.`total` AS `total`, `miss`.`miss` AS `miss`, (`total`.`total` - `miss`.`miss`) AS `matched` FROM total JOIN miss", "persist.type": "metric" } ] diff --git a/measure/src/test/resources/config-test2.json b/measure/src/test/resources/config-test2.json index 1a49d8b2a..2fa4953ad 100644 --- a/measure/src/test/resources/config-test2.json +++ b/measure/src/test/resources/config-test2.json @@ -1,5 +1,5 @@ { - "name": "accu batch test", + "name": "accu_batch_test", "process.type": "batch", @@ -38,18 +38,17 @@ "details": { "source": "source", "target": "target", - "miss.record": { - "name": "miss.record", + "miss.records": { + "name": "miss.records", "persist.type": "record" }, - "miss.count": { - "name": "miss.count", + "accuracy": { + "name": "accu", "persist.type": "metric" }, - "total.count": { - "name": "total.count", - "persist.type": "metric" - } + "miss": "miss", + "total": "total", + "matched": "matched" } } ] diff --git a/measure/src/test/resources/env-test.json b/measure/src/test/resources/env-test.json index 603fad8a1..e03d367c7 100644 --- a/measure/src/test/resources/env-test.json +++ b/measure/src/test/resources/env-test.json @@ -15,6 +15,14 @@ "config": { "max.log.lines": 100 } + }, + { + "type": "hdfs", + "config": { + "path": "hdfs://localhost/griffin/test/persist", + "max.persist.lines": 10000, + "max.lines.per.file": 10000 + } } ], diff --git a/measure/src/test/scala/org/apache/griffin/measure/process/JsonParseTest.scala b/measure/src/test/scala/org/apache/griffin/measure/process/JsonParseTest.scala index 9657078c4..0b3eb1003 100644 --- a/measure/src/test/scala/org/apache/griffin/measure/process/JsonParseTest.scala +++ b/measure/src/test/scala/org/apache/griffin/measure/process/JsonParseTest.scala @@ -95,7 +95,8 @@ class JsonParseTest extends FunSuite with Matchers with BeforeAndAfter with Logg // df1.show(10) // df1.registerTempTable("df1") val details = Map[String, Any](("df.name" -> "src")) - DataFrameOprs.fromJson(sqlContext, "df1", details) + val df1 = DataFrameOprs.fromJson(sqlContext, details) + df1.registerTempTable("df1") // 2. extract json array into lines // val rdd2 = df1.flatMap { row => From 2da5e8a708beada7b450398ba93e455ab9080aca Mon Sep 17 00:00:00 2001 From: Lionel Liu Date: Mon, 28 Aug 2017 15:35:13 +0800 Subject: [PATCH 056/111] v2 --- .../scala/org/apache/griffin/measure/utils/HdfsUtil.scala | 2 +- measure/src/test/resources/env-test.json | 8 -------- .../apache/griffin/measure/process/JsonParseTest.scala | 3 ++- 3 files changed, 3 insertions(+), 10 deletions(-) diff --git a/measure/src/main/scala/org/apache/griffin/measure/utils/HdfsUtil.scala b/measure/src/main/scala/org/apache/griffin/measure/utils/HdfsUtil.scala index 079359275..6dd54b7f6 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/utils/HdfsUtil.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/utils/HdfsUtil.scala @@ -27,7 +27,7 @@ object HdfsUtil { private val conf = new Configuration() conf.set("dfs.support.append", "true") - conf.set("fs.defaultFS", "hdfs://localhost") // debug @localhost +// conf.set("fs.defaultFS", "hdfs://localhost") // debug @localhost private val dfs = FileSystem.get(conf) diff --git a/measure/src/test/resources/env-test.json b/measure/src/test/resources/env-test.json index e03d367c7..603fad8a1 100644 --- a/measure/src/test/resources/env-test.json +++ b/measure/src/test/resources/env-test.json @@ -15,14 +15,6 @@ "config": { "max.log.lines": 100 } - }, - { - "type": "hdfs", - "config": { - "path": "hdfs://localhost/griffin/test/persist", - "max.persist.lines": 10000, - "max.lines.per.file": 10000 - } } ], diff --git a/measure/src/test/scala/org/apache/griffin/measure/process/JsonParseTest.scala b/measure/src/test/scala/org/apache/griffin/measure/process/JsonParseTest.scala index 0b3eb1003..04414a02a 100644 --- a/measure/src/test/scala/org/apache/griffin/measure/process/JsonParseTest.scala +++ b/measure/src/test/scala/org/apache/griffin/measure/process/JsonParseTest.scala @@ -27,7 +27,7 @@ import org.apache.griffin.measure.config.validator.AllParamValidator import org.apache.griffin.measure.log.Loggable import org.apache.griffin.measure.persist.PersistThreadPool import org.apache.griffin.measure.process.engine.DataFrameOprs -import org.apache.griffin.measure.utils.JsonUtil +import org.apache.griffin.measure.utils.{HdfsUtil, JsonUtil} import org.apache.hadoop.hive.ql.exec.UDF import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.sql.{Column, DataFrame, Row, SQLContext} @@ -218,6 +218,7 @@ class JsonParseTest extends FunSuite with Matchers with BeforeAndAfter with Logg // desc.show(100) // + } test ("json test 2") { From e8432fff67e8f4b027b74ca83dc43664ef4bcc17 Mon Sep 17 00:00:00 2001 From: Lionel Liu Date: Wed, 30 Aug 2017 09:30:07 +0800 Subject: [PATCH 057/111] test multi hdfs clusters --- .../griffin/measure/utils/HdfsUtilTest.scala | 132 ++++++++++++++++++ 1 file changed, 132 insertions(+) create mode 100644 measure/src/test/scala/org/apache/griffin/measure/utils/HdfsUtilTest.scala diff --git a/measure/src/test/scala/org/apache/griffin/measure/utils/HdfsUtilTest.scala b/measure/src/test/scala/org/apache/griffin/measure/utils/HdfsUtilTest.scala new file mode 100644 index 000000000..98d61819d --- /dev/null +++ b/measure/src/test/scala/org/apache/griffin/measure/utils/HdfsUtilTest.scala @@ -0,0 +1,132 @@ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ +package org.apache.griffin.measure.utils + +import java.io.{BufferedReader, FileReader, InputStreamReader} + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.{FileSystem, Path} +import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.hive.HiveContext +import org.apache.spark.{SparkConf, SparkContext} +import org.junit.runner.RunWith +import org.scalatest.junit.JUnitRunner +import org.scalatest.{BeforeAndAfter, FunSuite, Matchers} + +@RunWith(classOf[JUnitRunner]) +class HdfsUtilTest extends FunSuite with Matchers with BeforeAndAfter { + + private val seprator = "/" + + private val conf1 = new Configuration() + conf1.addResource(new Path("file:///apache/hadoop/etc/hadoop/core-site.xml")) + conf1.addResource(new Path("file:///apache/hadoop/etc/hadoop/hdfs-site.xml")) + private val dfs1 = FileSystem.get(conf1) + + private val conf2 = new Configuration() + conf2.addResource(new Path("file:///Users/lliu13/test/hadoop/core-site.xml")) + conf2.addResource(new Path("file:///Users/lliu13/test/hadoop/hdfs-site.xml")) + private val dfs2 = FileSystem.get(conf2) + + val conf = new SparkConf().setAppName("test_hdfs").setMaster("local[*]") + val sparkContext = new SparkContext(conf) + sparkContext.setLogLevel("WARN") + val sqlContext = new HiveContext(sparkContext) + + def listSubPaths(dfs: FileSystem, dirPath: String, subType: String, fullPath: Boolean = false): Iterable[String] = { + val path = new Path(dirPath) + try { + val fileStatusArray = dfs.listStatus(path) + fileStatusArray.filter { fileStatus => + subType match { + case "dir" => fileStatus.isDirectory + case "file" => fileStatus.isFile + case _ => true + } + }.map { fileStatus => + val fname = fileStatus.getPath.getName + if (fullPath) getHdfsFilePath(dirPath, fname) else fname + } + } catch { + case e: Throwable => { + println(s"list path files error: ${e.getMessage}") + Nil + } + } + } + + def getHdfsFilePath(parentPath: String, fileName: String): String = { + if (parentPath.endsWith(seprator)) parentPath + fileName else parentPath + seprator + fileName + } + + test ("test multiple hdfs") { + val list1 = listSubPaths(dfs1, "/", "dir", false) + println(list1) + + val list2 = listSubPaths(dfs2, "/", "dir", false) + println(list2) + + val path1 = "/depth/depth.bz2" + val istream1 = dfs1.open(new Path(path1)) + val reader1 = new BufferedReader(new InputStreamReader(istream1)) + val seq1 = scala.collection.mutable.MutableList[String]() + try { + var line = reader1.readLine() + while (line != null) { + val arr = line.split("\u0007") + seq1 ++= arr + line = reader1.readLine() + } + } finally { + reader1.close() + istream1.close() + } + +// val scanner = new java.util.Scanner(istream1,"UTF-8").useDelimiter("\u0007") +// val theString = if (scanner.hasNext()) scanner.next() else "" +// println(theString) +// scanner.close() + + println(seq1.size) + println(seq1.take(10)) + seq1.take(10).foreach(println) + +// val path2 = "/griffin/json/env.json" +// val istream2 = dfs2.open(new Path(path2)) +// val reader2 = new BufferedReader(new InputStreamReader(istream2)) +// val seq2 = scala.collection.mutable.MutableList[String]() +// try { +// var line = reader2.readLine() +// while (line != null) { +// line = reader2.readLine() +// seq2 += line +// } +// } catch { +// case e: Throwable => { +// println("error in reading") +// } +// } finally { +// reader2.close() +// istream2.close() +// } +// println(seq2.size) +// println(seq2.take(10)) + } + +} From 586749b7b12f1c018e2ed86e8409f19433afff5b Mon Sep 17 00:00:00 2001 From: Lionel Liu Date: Wed, 30 Aug 2017 18:06:51 +0800 Subject: [PATCH 058/111] profiling simple --- .../rules/adaptor/GriffinDslAdaptor.scala | 118 ++++++--- .../rules/dsl/analyzer/AccuracyAnalyzer.scala | 6 +- .../rules/dsl/analyzer/BasicAnalyzer.scala | 9 + .../dsl/analyzer/ProfilingAnalyzer.scala | 35 +++ .../rules/dsl/expr/AliasableExpr.scala | 25 ++ .../griffin/measure/rules/dsl/expr/Expr.scala | 2 + .../measure/rules/dsl/expr/Expressions.scala | 27 ++ .../measure/rules/dsl/expr/FunctionExpr.scala | 4 +- .../measure/rules/dsl/expr/LogicalExpr.scala | 16 +- .../measure/rules/dsl/expr/MathExpr.scala | 16 +- .../measure/rules/dsl/expr/SelectExpr.scala | 7 +- .../rules/dsl/parser/BasicParser.scala | 29 +- .../rules/dsl/parser/GriffinDslParser.scala | 15 +- .../test/resources/config-test-profiling.json | 37 +++ measure/src/test/resources/config-test2.json | 2 +- .../measure/process/BatchProcessTest.scala | 3 +- .../rules/dsl/parser/BasicParserTest.scala | 1 + .../griffin/measure/utils/HdfsUtilTest.scala | 250 +++++++++--------- 18 files changed, 414 insertions(+), 188 deletions(-) create mode 100644 measure/src/main/scala/org/apache/griffin/measure/rules/dsl/analyzer/ProfilingAnalyzer.scala create mode 100644 measure/src/main/scala/org/apache/griffin/measure/rules/dsl/expr/AliasableExpr.scala create mode 100644 measure/src/main/scala/org/apache/griffin/measure/rules/dsl/expr/Expressions.scala create mode 100644 measure/src/test/resources/config-test-profiling.json diff --git a/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/GriffinDslAdaptor.scala b/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/GriffinDslAdaptor.scala index fa1a5bcf6..1d4146dd0 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/GriffinDslAdaptor.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/GriffinDslAdaptor.scala @@ -21,7 +21,7 @@ package org.apache.griffin.measure.rules.adaptor import org.apache.griffin.measure.config.params.user.RuleParam import org.apache.griffin.measure.rules.dsl._ import org.apache.griffin.measure.rules.dsl.analyzer._ -import org.apache.griffin.measure.rules.dsl.expr.Expr +import org.apache.griffin.measure.rules.dsl.expr._ import org.apache.griffin.measure.rules.dsl.parser.GriffinDslParser import org.apache.griffin.measure.rules.step._ @@ -41,19 +41,24 @@ case class GriffinDslAdaptor(dataSourceNames: Seq[String], functionNames: Seq[St val _Miss = "miss" val _Total = "total" val _Matched = "matched" - def getNameOpt(param: Map[String, Any], key: String): Option[String] = param.get(key).flatMap(a => Some(a.toString)) - def resultName(param: Map[String, Any], key: String): String = { - val nameOpt = param.get(key) match { - case Some(prm: Map[String, Any]) => StepInfo.getNameOpt(prm) - case _ => None - } - nameOpt.getOrElse(key) + } + object ProfilingInfo { + val _Source = "source" + val _Profiling = "profiling" + } + + def getNameOpt(param: Map[String, Any], key: String): Option[String] = param.get(key).flatMap(a => Some(a.toString)) + def resultName(param: Map[String, Any], key: String): String = { + val nameOpt = param.get(key) match { + case Some(prm: Map[String, Any]) => StepInfo.getNameOpt(prm) + case _ => None } - def resultPersistType(param: Map[String, Any], key: String, defPersistType: PersistType): PersistType = { - param.get(key) match { - case Some(prm: Map[String, Any]) => StepInfo.getPersistType(prm) - case _ => defPersistType - } + nameOpt.getOrElse(key) + } + def resultPersistType(param: Map[String, Any], key: String, defPersistType: PersistType): PersistType = { + param.get(key) match { + case Some(prm: Map[String, Any]) => StepInfo.getPersistType(prm) + case _ => defPersistType } } @@ -77,12 +82,14 @@ case class GriffinDslAdaptor(dataSourceNames: Seq[String], functionNames: Seq[St dqType match { case AccuracyType => { Seq[String]( - AccuracyInfo.resultName(param, AccuracyInfo._MissRecords), - AccuracyInfo.resultName(param, AccuracyInfo._Accuracy) + resultName(param, AccuracyInfo._MissRecords), + resultName(param, AccuracyInfo._Accuracy) ) } case ProfilingType => { - Nil + Seq[String]( + resultName(param, ProfilingInfo._Profiling) + ) } case TimelinessType => { Nil @@ -96,12 +103,12 @@ case class GriffinDslAdaptor(dataSourceNames: Seq[String], functionNames: Seq[St def adaptConcreteRuleStep(ruleStep: RuleStep): Seq[ConcreteRuleStep] = { ruleStep match { - case rs @ GriffinDslStep(_, rule, _, _) => { + case rs @ GriffinDslStep(_, rule, dqType, _) => { val exprOpt = try { - val result = parser.parseAll(parser.rootExpression, rule) + val result = parser.parseRule(rule, dqType) if (result.successful) Some(result.get) else { - warn(s"adapt concrete rule step warn: ${rule}") + warn(s"adapt concrete rule step warn: parse rule [ ${rule} ] fails") None } } catch { @@ -113,7 +120,14 @@ case class GriffinDslAdaptor(dataSourceNames: Seq[String], functionNames: Seq[St exprOpt match { case Some(expr) => { - transConcreteRuleSteps(rs, expr) + try { + transConcreteRuleSteps(rs, expr) + } catch { + case e: Throwable => { + error(s"trans concrete rule step error: ${e.getMessage}") + Nil + } + } } case _ => Nil } @@ -126,25 +140,26 @@ case class GriffinDslAdaptor(dataSourceNames: Seq[String], functionNames: Seq[St val details = ruleStep.details ruleStep.dqType match { case AccuracyType => { - val sourceName = AccuracyInfo.getNameOpt(details, AccuracyInfo._Source) match { + val sourceName = getNameOpt(details, AccuracyInfo._Source) match { case Some(name) => name case _ => dataSourceNames.head } - val targetName = AccuracyInfo.getNameOpt(details, AccuracyInfo._Target) match { + val targetName = getNameOpt(details, AccuracyInfo._Target) match { case Some(name) => name case _ => dataSourceNames.tail.head } - val analyzer = AccuracyAnalyzer(expr, sourceName, targetName) + val analyzer = AccuracyAnalyzer(expr.asInstanceOf[LogicalExpr], sourceName, targetName) // 1. miss record val missRecordsSql = { - val selClause = analyzer.sourceSelectionExprs.map { sel => - val alias = sel.alias match { - case Some(a) => s" AS ${a}" - case _ => "" - } - s"${sel.desc}${alias}" - }.mkString(", ") +// val selClause = analyzer.selectionExprs.map { sel => +// val alias = sel.alias match { +// case Some(a) => s" AS ${a}" +// case _ => "" +// } +// s"${sel.desc}${alias}" +// }.mkString(", ") + val selClause = s"`${sourceName}`.*" val onClause = expr.coalesceDesc @@ -156,18 +171,18 @@ case class GriffinDslAdaptor(dataSourceNames: Seq[String], functionNames: Seq[St }.mkString(" AND ") val whereClause = s"(NOT (${sourceIsNull})) AND (${targetIsNull})" - s"SELECT ${selClause} FROM ${sourceName} LEFT JOIN ${targetName} ON ${onClause} WHERE ${whereClause}" + s"SELECT ${selClause} FROM `${sourceName}` LEFT JOIN `${targetName}` ON ${onClause} WHERE ${whereClause}" } - val missRecordsName = AccuracyInfo.resultName(details, AccuracyInfo._MissRecords) + val missRecordsName = resultName(details, AccuracyInfo._MissRecords) val missRecordsStep = SparkSqlStep( missRecordsName, missRecordsSql, - AccuracyInfo.resultPersistType(details, AccuracyInfo._MissRecords, RecordPersistType) + resultPersistType(details, AccuracyInfo._MissRecords, RecordPersistType) ) // 2. miss count val missTableName = "_miss_" - val missColName = AccuracyInfo.getNameOpt(details, AccuracyInfo._Miss).getOrElse(AccuracyInfo._Miss) + val missColName = getNameOpt(details, AccuracyInfo._Miss).getOrElse(AccuracyInfo._Miss) val missSql = { s"SELECT COUNT(*) AS `${missColName}` FROM `${missRecordsName}`" } @@ -179,7 +194,7 @@ case class GriffinDslAdaptor(dataSourceNames: Seq[String], functionNames: Seq[St // 3. total count val totalTableName = "_total_" - val totalColName = AccuracyInfo.getNameOpt(details, AccuracyInfo._Total).getOrElse(AccuracyInfo._Total) + val totalColName = getNameOpt(details, AccuracyInfo._Total).getOrElse(AccuracyInfo._Total) val totalSql = { s"SELECT COUNT(*) AS `${totalColName}` FROM `${sourceName}`" } @@ -190,7 +205,7 @@ case class GriffinDslAdaptor(dataSourceNames: Seq[String], functionNames: Seq[St ) // 4. accuracy metric - val matchedColName = AccuracyInfo.getNameOpt(details, AccuracyInfo._Matched).getOrElse(AccuracyInfo._Matched) + val matchedColName = getNameOpt(details, AccuracyInfo._Matched).getOrElse(AccuracyInfo._Matched) val accuracyMetricSql = { s""" |SELECT `${totalTableName}`.`${totalColName}` AS `${totalColName}`, @@ -199,17 +214,42 @@ case class GriffinDslAdaptor(dataSourceNames: Seq[String], functionNames: Seq[St |FROM `${totalTableName}` JOIN `${missTableName}` """.stripMargin } - val accuracyMetricName = AccuracyInfo.resultName(details, AccuracyInfo._Accuracy) + val accuracyMetricName = resultName(details, AccuracyInfo._Accuracy) val accuracyMetricStep = SparkSqlStep( accuracyMetricName, accuracyMetricSql, - AccuracyInfo.resultPersistType(details, AccuracyInfo._Accuracy, MetricPersistType) + resultPersistType(details, AccuracyInfo._Accuracy, MetricPersistType) ) missRecordsStep :: missStep :: totalStep :: accuracyMetricStep :: Nil } case ProfilingType => { - Nil + val sourceName = getNameOpt(details, ProfilingInfo._Source) match { + case Some(name) => name + case _ => dataSourceNames.head + } + val analyzer = ProfilingAnalyzer(expr.asInstanceOf[Expressions], sourceName) + + // 1. select statement + val profilingSql = { + val selClause = analyzer.selectionExprs.map { sel => + val alias = sel match { + case s: AliasableExpr if (s.alias.nonEmpty) => s" AS ${s.alias.get}" + case _ => "" + } + s"${sel.desc}${alias}" + }.mkString(", ") + + s"SELECT ${selClause} FROM ${sourceName}" + } + val profilingMetricName = resultName(details, ProfilingInfo._Profiling) + val profilingStep = SparkSqlStep( + profilingMetricName, + profilingSql, + resultPersistType(details, ProfilingInfo._Profiling, RecordPersistType) + ) + + profilingStep :: Nil } case TimelinessType => { Nil diff --git a/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/analyzer/AccuracyAnalyzer.scala b/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/analyzer/AccuracyAnalyzer.scala index bc5403015..1071a967b 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/analyzer/AccuracyAnalyzer.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/analyzer/AccuracyAnalyzer.scala @@ -21,7 +21,7 @@ package org.apache.griffin.measure.rules.dsl.analyzer import org.apache.griffin.measure.rules.dsl.expr._ -case class AccuracyAnalyzer(expr: Expr, sourceName: String, targetName: String) extends BasicAnalyzer { +case class AccuracyAnalyzer(expr: LogicalExpr, sourceName: String, targetName: String) extends BasicAnalyzer { val dataSourceNames = expr.preOrderTraverseDepthFirst(Set[String]())(seqDataSourceNames, combDataSourceNames) @@ -34,4 +34,8 @@ case class AccuracyAnalyzer(expr: Expr, sourceName: String, targetName: String) expr.preOrderTraverseDepthFirst(Seq[SelectionExpr]())(seq, combSelectionExprs) } + val selectionExprs = sourceSelectionExprs ++ { + expr.preOrderTraverseDepthFirst(Seq[AliasableExpr]())(seqWithAliasExprs, combWithAliasExprs) + } + } diff --git a/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/analyzer/BasicAnalyzer.scala b/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/analyzer/BasicAnalyzer.scala index 21192c79d..94516ce35 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/analyzer/BasicAnalyzer.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/analyzer/BasicAnalyzer.scala @@ -41,4 +41,13 @@ trait BasicAnalyzer extends Serializable { } val combSelectionExprs = (a: Seq[SelectionExpr], b: Seq[SelectionExpr]) => a ++ b + val seqWithAliasExprs = (expr: Expr, v: Seq[AliasableExpr]) => { + expr match { + case se: SelectExpr => v + case a: AliasableExpr if (a.alias.nonEmpty) => v :+ a + case _ => v + } + } + val combWithAliasExprs = (a: Seq[AliasableExpr], b: Seq[AliasableExpr]) => a ++ b + } diff --git a/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/analyzer/ProfilingAnalyzer.scala b/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/analyzer/ProfilingAnalyzer.scala new file mode 100644 index 000000000..dfa3df4fd --- /dev/null +++ b/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/analyzer/ProfilingAnalyzer.scala @@ -0,0 +1,35 @@ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ +package org.apache.griffin.measure.rules.dsl.analyzer + +import org.apache.griffin.measure.rules.dsl.expr._ + + +case class ProfilingAnalyzer(expr: Expressions, sourceName: String) extends BasicAnalyzer { + + val dataSourceNames = expr.preOrderTraverseDepthFirst(Set[String]())(seqDataSourceNames, combDataSourceNames) + + val sourceSelectionExprs = { + val seq = seqSelectionExprs(sourceName) + expr.preOrderTraverseDepthFirst(Seq[SelectionExpr]())(seq, combSelectionExprs) + } + + val selectionExprs = expr.exprs.map(_.extractSelf) + +} diff --git a/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/expr/AliasableExpr.scala b/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/expr/AliasableExpr.scala new file mode 100644 index 000000000..b6ad00a7c --- /dev/null +++ b/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/expr/AliasableExpr.scala @@ -0,0 +1,25 @@ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ +package org.apache.griffin.measure.rules.dsl.expr + +trait AliasableExpr extends Expr { + + def alias: Option[String] + +} diff --git a/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/expr/Expr.scala b/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/expr/Expr.scala index 5f7d51232..64b02ff6b 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/expr/Expr.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/expr/Expr.scala @@ -24,4 +24,6 @@ trait Expr extends TreeNode with Serializable { def coalesceDesc: String + def extractSelf: Expr = this + } diff --git a/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/expr/Expressions.scala b/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/expr/Expressions.scala new file mode 100644 index 000000000..81e35dfa4 --- /dev/null +++ b/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/expr/Expressions.scala @@ -0,0 +1,27 @@ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ +package org.apache.griffin.measure.rules.dsl.expr + +case class Expressions(exprs: Seq[Expr]) extends Expr { + + addChildren(exprs) + + def desc: String = s"${exprs.map(_.desc).mkString(", ")}" + def coalesceDesc: String = s"${exprs.map(_.coalesceDesc).mkString(", ")}" +} \ No newline at end of file diff --git a/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/expr/FunctionExpr.scala b/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/expr/FunctionExpr.scala index 272291e60..910e36452 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/expr/FunctionExpr.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/expr/FunctionExpr.scala @@ -18,10 +18,12 @@ under the License. */ package org.apache.griffin.measure.rules.dsl.expr -case class FunctionExpr(functionName: String, args: Seq[Expr]) extends Expr { +case class FunctionExpr(functionName: String, args: Seq[Expr], aliasOpt: Option[String] + ) extends Expr with AliasableExpr { addChildren(args) def desc: String = s"${functionName}(${args.map(_.desc).mkString(", ")})" def coalesceDesc: String = desc + def alias: Option[String] = if (aliasOpt.isEmpty) Some(functionName) else aliasOpt } \ No newline at end of file diff --git a/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/expr/LogicalExpr.scala b/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/expr/LogicalExpr.scala index 1df88c96c..824db19c7 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/expr/LogicalExpr.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/expr/LogicalExpr.scala @@ -98,12 +98,18 @@ case class IsNanExpr(head: Expr, is: Boolean) extends LogicalExpr { // ----------- -case class LogicalFactorExpr(factor: Expr, withBracket: Boolean) extends LogicalExpr { +case class LogicalFactorExpr(factor: Expr, withBracket: Boolean, aliasOpt: Option[String] + ) extends LogicalExpr with AliasableExpr { addChild(factor) def desc: String = if (withBracket) s"(${factor.desc})" else factor.desc def coalesceDesc: String = factor.coalesceDesc + def alias: Option[String] = aliasOpt + override def extractSelf: Expr = { + if (aliasOpt.nonEmpty) this + else factor.extractSelf + } } case class UnaryLogicalExpr(oprs: Seq[String], factor: LogicalExpr) extends LogicalExpr { @@ -126,6 +132,10 @@ case class UnaryLogicalExpr(oprs: Seq[String], factor: LogicalExpr) extends Logi case _ => s.toUpperCase } } + override def extractSelf: Expr = { + if (oprs.nonEmpty) this + else factor.extractSelf + } } case class BinaryLogicalExpr(factor: LogicalExpr, tails: Seq[(String, LogicalExpr)]) extends LogicalExpr { @@ -153,4 +163,8 @@ case class BinaryLogicalExpr(factor: LogicalExpr, tails: Seq[(String, LogicalExp case _ => s.toUpperCase } } + override def extractSelf: Expr = { + if (tails.nonEmpty) this + else factor.extractSelf + } } \ No newline at end of file diff --git a/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/expr/MathExpr.scala b/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/expr/MathExpr.scala index 668d2cef7..9d4e54d64 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/expr/MathExpr.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/expr/MathExpr.scala @@ -21,12 +21,18 @@ package org.apache.griffin.measure.rules.dsl.expr trait MathExpr extends Expr { } -case class MathFactorExpr(factor: Expr, withBracket: Boolean) extends MathExpr { +case class MathFactorExpr(factor: Expr, withBracket: Boolean, aliasOpt: Option[String] + ) extends MathExpr with AliasableExpr { addChild(factor) def desc: String = if (withBracket) s"(${factor.desc})" else factor.desc def coalesceDesc: String = factor.coalesceDesc + def alias: Option[String] = aliasOpt + override def extractSelf: Expr = { + if (aliasOpt.nonEmpty) this + else factor.extractSelf + } } case class UnaryMathExpr(oprs: Seq[String], factor: MathExpr) extends MathExpr { @@ -43,6 +49,10 @@ case class UnaryMathExpr(oprs: Seq[String], factor: MathExpr) extends MathExpr { s"(${opr}${fac})" } } + override def extractSelf: Expr = { + if (oprs.nonEmpty) this + else factor.extractSelf + } } case class BinaryMathExpr(factor: MathExpr, tails: Seq[(String, MathExpr)]) extends MathExpr { @@ -63,4 +73,8 @@ case class BinaryMathExpr(factor: MathExpr, tails: Seq[(String, MathExpr)]) exte } if (tails.size <= 0) res else s"${res}" } + override def extractSelf: Expr = { + if (tails.nonEmpty) this + else factor.extractSelf + } } \ No newline at end of file diff --git a/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/expr/SelectExpr.scala b/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/expr/SelectExpr.scala index 1273e0302..f4fc92d05 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/expr/SelectExpr.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/expr/SelectExpr.scala @@ -37,10 +37,7 @@ case class OtherHeadExpr(expr: Expr) extends HeadExpr { // ------------- -trait SelectExpr extends Expr { - - def alias: Option[String] - +trait SelectExpr extends Expr with AliasableExpr { } case class FieldSelectExpr(field: String) extends SelectExpr { @@ -64,7 +61,7 @@ case class FunctionSelectExpr(functionName: String, args: Seq[Expr]) extends Sel def desc: String = "" def coalesceDesc: String = desc - def alias: Option[String] = None + def alias: Option[String] = Some(functionName) } // ------------- diff --git a/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/parser/BasicParser.scala b/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/parser/BasicParser.scala index 2d7666f1c..ba7d1fcbf 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/parser/BasicParser.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/parser/BasicParser.scala @@ -24,7 +24,8 @@ import scala.util.parsing.combinator.JavaTokenParsers trait BasicParser extends JavaTokenParsers with Serializable { - def rootExpression = expression + val dataSourceNames: Seq[String] + val functionNames: Seq[String] /** * BNF for basic parser @@ -77,9 +78,6 @@ trait BasicParser extends JavaTokenParsers with Serializable { * = */ - val dataSourceNames: Seq[String] - val functionNames: Seq[String] - protected def genNamesParser(names: Seq[String]): Parser[String] = { names.map { fn => s"""${fn}""".r: Parser[String] @@ -190,9 +188,9 @@ trait BasicParser extends JavaTokenParsers with Serializable { */ def mathFactor: Parser[MathExpr] = (literal | function | selection) ^^ { - MathFactorExpr(_, false) - } | LBR ~> mathExpression <~ RBR ^^ { - MathFactorExpr(_, true) + MathFactorExpr(_, false, None) + } | LBR ~ mathExpression ~ RBR ~ opt(asAlias) ^^ { + case _ ~ expr ~ _ ~ aliasOpt => MathFactorExpr(expr, true, aliasOpt) } def unaryMathExpression: Parser[MathExpr] = rep(MATH_UNARY) ~ mathFactor ^^ { case Nil ~ a => a @@ -243,9 +241,9 @@ trait BasicParser extends JavaTokenParsers with Serializable { } def logicalFactor: Parser[LogicalExpr] = (inExpr | betweenExpr | likeExpr | isNullExpr | isNanExpr | mathExpression) ^^ { - LogicalFactorExpr(_, false) - } | LBR ~> logicalExpression <~ RBR ^^ { - LogicalFactorExpr(_, true) + LogicalFactorExpr(_, false, None) + } | LBR ~ logicalExpression ~ RBR ~ opt(asAlias) ^^ { + case _ ~ expr ~ _ ~ aliasOpt => LogicalFactorExpr(expr, true, aliasOpt) } def unaryLogicalExpression: Parser[LogicalExpr] = rep(LOGICAL_UNARY) ~ logicalFactor ^^ { case Nil ~ a => a @@ -276,9 +274,16 @@ trait BasicParser extends JavaTokenParsers with Serializable { * ::= */ - def function: Parser[FunctionExpr] = FunctionName ~ LBR ~ repsep(argument, COMMA) ~ RBR ^^ { - case name ~ _ ~ args ~ _ => FunctionExpr(name, args) + def function: Parser[FunctionExpr] = FunctionName ~ LBR ~ repsep(argument, COMMA) ~ RBR ~ opt(asAlias) ^^ { + case name ~ _ ~ args ~ _ ~ aliasOpt => FunctionExpr(name, args, aliasOpt) } def argument: Parser[Expr] = expression + /** + * -- exprs -- + * = [, ]* + */ + + def expressions: Parser[Expressions] = rep1sep(expression, COMMA) ^^ { Expressions(_) } + } diff --git a/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/parser/GriffinDslParser.scala b/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/parser/GriffinDslParser.scala index 5ab1a11e2..987986249 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/parser/GriffinDslParser.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/parser/GriffinDslParser.scala @@ -18,6 +18,19 @@ under the License. */ package org.apache.griffin.measure.rules.dsl.parser -case class GriffinDslParser(dataSourceNames: Seq[String], functionNames: Seq[String]) extends BasicParser { +import org.apache.griffin.measure.rules.dsl._ +import org.apache.griffin.measure.rules.dsl.expr.Expr + +case class GriffinDslParser(dataSourceNames: Seq[String], functionNames: Seq[String] + ) extends BasicParser { + + def parseRule(rule: String, dqType: DqType): ParseResult[Expr] = { + val rootExpr = dqType match { + case AccuracyType => logicalExpression + case ProfilingType => expressions + case _ => expression + } + parseAll(rootExpr, rule) + } } diff --git a/measure/src/test/resources/config-test-profiling.json b/measure/src/test/resources/config-test-profiling.json new file mode 100644 index 000000000..201a3d4dd --- /dev/null +++ b/measure/src/test/resources/config-test-profiling.json @@ -0,0 +1,37 @@ +{ + "name": "prof_batch_test", + + "process.type": "batch", + + "data.sources": [ + { + "name": "source", + "connectors": [ + { + "type": "avro", + "version": "1.7", + "config": { + "file.name": "src/test/resources/users_info_src.avro" + } + } + ] + } + ], + + "evaluateRule": { + "rules": [ + { + "dsl.type": "griffin-dsl", + "dq.type": "profiling", + "rule": "source.user_id.max(), source.first_name.count() as cnt", + "details": { + "source": "source", + "profiling": { + "name": "prof", + "persist.type": "metric" + } + } + } + ] + } +} \ No newline at end of file diff --git a/measure/src/test/resources/config-test2.json b/measure/src/test/resources/config-test2.json index 2fa4953ad..4940a72ff 100644 --- a/measure/src/test/resources/config-test2.json +++ b/measure/src/test/resources/config-test2.json @@ -34,7 +34,7 @@ { "dsl.type": "griffin-dsl", "dq.type": "accuracy", - "rule": "(source.user_id as id) = target.user_id AND source.first_name = 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", + "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", "target": "target", diff --git a/measure/src/test/scala/org/apache/griffin/measure/process/BatchProcessTest.scala b/measure/src/test/scala/org/apache/griffin/measure/process/BatchProcessTest.scala index 1ac9249ca..11fa21d2d 100644 --- a/measure/src/test/scala/org/apache/griffin/measure/process/BatchProcessTest.scala +++ b/measure/src/test/scala/org/apache/griffin/measure/process/BatchProcessTest.scala @@ -36,7 +36,8 @@ import scala.util.{Failure, Success, Try} class BatchProcessTest extends FunSuite with Matchers with BeforeAndAfter with Loggable { val envFile = "src/test/resources/env-test.json" - val confFile = "src/test/resources/config-test2.json" + val confFile = "src/test/resources/config-test-profiling.json" +// val confFile = "src/test/resources/config-test2.json" val envFsType = "local" val userFsType = "local" diff --git a/measure/src/test/scala/org/apache/griffin/measure/rules/dsl/parser/BasicParserTest.scala b/measure/src/test/scala/org/apache/griffin/measure/rules/dsl/parser/BasicParserTest.scala index 13f230757..1d267985f 100644 --- a/measure/src/test/scala/org/apache/griffin/measure/rules/dsl/parser/BasicParserTest.scala +++ b/measure/src/test/scala/org/apache/griffin/measure/rules/dsl/parser/BasicParserTest.scala @@ -31,6 +31,7 @@ class BasicParserTest extends FunSuite with Matchers with BeforeAndAfter { val parser = new BasicParser{ val dataSourceNames: Seq[String] = "source" :: "target" :: Nil val functionNames: Seq[String] = "func" :: "get_json_object" :: Nil + def rootExpression: Parser[Expr] = expression } test("test literal") { diff --git a/measure/src/test/scala/org/apache/griffin/measure/utils/HdfsUtilTest.scala b/measure/src/test/scala/org/apache/griffin/measure/utils/HdfsUtilTest.scala index 98d61819d..e360c1dfe 100644 --- a/measure/src/test/scala/org/apache/griffin/measure/utils/HdfsUtilTest.scala +++ b/measure/src/test/scala/org/apache/griffin/measure/utils/HdfsUtilTest.scala @@ -1,132 +1,132 @@ -/* -Licensed to the Apache Software Foundation (ASF) under one -or more contributor license agreements. See the NOTICE file -distributed with this work for additional information -regarding copyright ownership. The ASF licenses this file -to you under the Apache License, Version 2.0 (the -"License"); you may not use this file except in compliance -with the License. You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, -software distributed under the License is distributed on an -"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -KIND, either express or implied. See the License for the -specific language governing permissions and limitations -under the License. -*/ -package org.apache.griffin.measure.utils - -import java.io.{BufferedReader, FileReader, InputStreamReader} - -import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.{FileSystem, Path} -import org.apache.spark.sql.SQLContext -import org.apache.spark.sql.hive.HiveContext -import org.apache.spark.{SparkConf, SparkContext} -import org.junit.runner.RunWith -import org.scalatest.junit.JUnitRunner -import org.scalatest.{BeforeAndAfter, FunSuite, Matchers} - -@RunWith(classOf[JUnitRunner]) -class HdfsUtilTest extends FunSuite with Matchers with BeforeAndAfter { - - private val seprator = "/" - - private val conf1 = new Configuration() - conf1.addResource(new Path("file:///apache/hadoop/etc/hadoop/core-site.xml")) - conf1.addResource(new Path("file:///apache/hadoop/etc/hadoop/hdfs-site.xml")) - private val dfs1 = FileSystem.get(conf1) - - private val conf2 = new Configuration() - conf2.addResource(new Path("file:///Users/lliu13/test/hadoop/core-site.xml")) - conf2.addResource(new Path("file:///Users/lliu13/test/hadoop/hdfs-site.xml")) - private val dfs2 = FileSystem.get(conf2) - - val conf = new SparkConf().setAppName("test_hdfs").setMaster("local[*]") - val sparkContext = new SparkContext(conf) - sparkContext.setLogLevel("WARN") - val sqlContext = new HiveContext(sparkContext) - - def listSubPaths(dfs: FileSystem, dirPath: String, subType: String, fullPath: Boolean = false): Iterable[String] = { - val path = new Path(dirPath) - try { - val fileStatusArray = dfs.listStatus(path) - fileStatusArray.filter { fileStatus => - subType match { - case "dir" => fileStatus.isDirectory - case "file" => fileStatus.isFile - case _ => true - } - }.map { fileStatus => - val fname = fileStatus.getPath.getName - if (fullPath) getHdfsFilePath(dirPath, fname) else fname - } - } catch { - case e: Throwable => { - println(s"list path files error: ${e.getMessage}") - Nil - } - } - } - - def getHdfsFilePath(parentPath: String, fileName: String): String = { - if (parentPath.endsWith(seprator)) parentPath + fileName else parentPath + seprator + fileName - } - - test ("test multiple hdfs") { - val list1 = listSubPaths(dfs1, "/", "dir", false) - println(list1) - - val list2 = listSubPaths(dfs2, "/", "dir", false) - println(list2) - - val path1 = "/depth/depth.bz2" - val istream1 = dfs1.open(new Path(path1)) - val reader1 = new BufferedReader(new InputStreamReader(istream1)) - val seq1 = scala.collection.mutable.MutableList[String]() - try { - var line = reader1.readLine() - while (line != null) { - val arr = line.split("\u0007") - seq1 ++= arr - line = reader1.readLine() - } - } finally { - reader1.close() - istream1.close() - } - -// val scanner = new java.util.Scanner(istream1,"UTF-8").useDelimiter("\u0007") -// val theString = if (scanner.hasNext()) scanner.next() else "" -// println(theString) -// scanner.close() - - println(seq1.size) - println(seq1.take(10)) - seq1.take(10).foreach(println) - -// val path2 = "/griffin/json/env.json" -// val istream2 = dfs2.open(new Path(path2)) -// val reader2 = new BufferedReader(new InputStreamReader(istream2)) -// val seq2 = scala.collection.mutable.MutableList[String]() +///* +//Licensed to the Apache Software Foundation (ASF) under one +//or more contributor license agreements. See the NOTICE file +//distributed with this work for additional information +//regarding copyright ownership. The ASF licenses this file +//to you under the Apache License, Version 2.0 (the +//"License"); you may not use this file except in compliance +//with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +//Unless required by applicable law or agreed to in writing, +//software distributed under the License is distributed on an +//"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +//KIND, either express or implied. See the License for the +//specific language governing permissions and limitations +//under the License. +//*/ +//package org.apache.griffin.measure.utils +// +//import java.io.{BufferedReader, FileReader, InputStreamReader} +// +//import org.apache.hadoop.conf.Configuration +//import org.apache.hadoop.fs.{FileSystem, Path} +//import org.apache.spark.sql.SQLContext +//import org.apache.spark.sql.hive.HiveContext +//import org.apache.spark.{SparkConf, SparkContext} +//import org.junit.runner.RunWith +//import org.scalatest.junit.JUnitRunner +//import org.scalatest.{BeforeAndAfter, FunSuite, Matchers} +// +//@RunWith(classOf[JUnitRunner]) +//class HdfsUtilTest extends FunSuite with Matchers with BeforeAndAfter { +// +// private val seprator = "/" +// +// private val conf1 = new Configuration() +// conf1.addResource(new Path("file:///apache/hadoop/etc/hadoop/core-site.xml")) +// conf1.addResource(new Path("file:///apache/hadoop/etc/hadoop/hdfs-site.xml")) +// private val dfs1 = FileSystem.get(conf1) +// +// private val conf2 = new Configuration() +// conf2.addResource(new Path("file:///Users/lliu13/test/hadoop/core-site.xml")) +// conf2.addResource(new Path("file:///Users/lliu13/test/hadoop/hdfs-site.xml")) +// private val dfs2 = FileSystem.get(conf2) +// +// val conf = new SparkConf().setAppName("test_hdfs").setMaster("local[*]") +// val sparkContext = new SparkContext(conf) +// sparkContext.setLogLevel("WARN") +// val sqlContext = new HiveContext(sparkContext) +// +// def listSubPaths(dfs: FileSystem, dirPath: String, subType: String, fullPath: Boolean = false): Iterable[String] = { +// val path = new Path(dirPath) // try { -// var line = reader2.readLine() -// while (line != null) { -// line = reader2.readLine() -// seq2 += line +// val fileStatusArray = dfs.listStatus(path) +// fileStatusArray.filter { fileStatus => +// subType match { +// case "dir" => fileStatus.isDirectory +// case "file" => fileStatus.isFile +// case _ => true +// } +// }.map { fileStatus => +// val fname = fileStatus.getPath.getName +// if (fullPath) getHdfsFilePath(dirPath, fname) else fname // } // } catch { // case e: Throwable => { -// println("error in reading") +// println(s"list path files error: ${e.getMessage}") +// Nil +// } +// } +// } +// +// def getHdfsFilePath(parentPath: String, fileName: String): String = { +// if (parentPath.endsWith(seprator)) parentPath + fileName else parentPath + seprator + fileName +// } +// +// test ("test multiple hdfs") { +// val list1 = listSubPaths(dfs1, "/", "dir", false) +// println(list1) +// +// val list2 = listSubPaths(dfs2, "/", "dir", false) +// println(list2) +// +// val path1 = "/depth/depth.bz2" +// val istream1 = dfs1.open(new Path(path1)) +// val reader1 = new BufferedReader(new InputStreamReader(istream1)) +// val seq1 = scala.collection.mutable.MutableList[String]() +// try { +// var line = reader1.readLine() +// while (line != null) { +// val arr = line.split("\u0007") +// seq1 ++= arr +// line = reader1.readLine() // } // } finally { -// reader2.close() -// istream2.close() +// reader1.close() +// istream1.close() // } -// println(seq2.size) -// println(seq2.take(10)) - } - -} +// +//// val scanner = new java.util.Scanner(istream1,"UTF-8").useDelimiter("\u0007") +//// val theString = if (scanner.hasNext()) scanner.next() else "" +//// println(theString) +//// scanner.close() +// +// println(seq1.size) +// println(seq1.take(10)) +// seq1.take(10).foreach(println) +// +//// val path2 = "/griffin/json/env.json" +//// val istream2 = dfs2.open(new Path(path2)) +//// val reader2 = new BufferedReader(new InputStreamReader(istream2)) +//// val seq2 = scala.collection.mutable.MutableList[String]() +//// try { +//// var line = reader2.readLine() +//// while (line != null) { +//// line = reader2.readLine() +//// seq2 += line +//// } +//// } catch { +//// case e: Throwable => { +//// println("error in reading") +//// } +//// } finally { +//// reader2.close() +//// istream2.close() +//// } +//// println(seq2.size) +//// println(seq2.take(10)) +// } +// +//} From a3807cca64c162756c3f0dcfff7188610f04d5cc Mon Sep 17 00:00:00 2001 From: Lionel Liu Date: Thu, 31 Aug 2017 14:00:56 +0800 Subject: [PATCH 059/111] result collect --- .../griffin/measure/persist/HdfsPersist.scala | 53 ++++---- .../griffin/measure/persist/HttpPersist.scala | 30 +++-- .../measure/persist/LoggerPersist.scala | 28 ++-- .../measure/persist/MultiPersists.scala | 3 +- .../griffin/measure/persist/Persist.scala | 3 +- .../measure/process/BatchDqProcess.scala | 2 +- .../process/engine/DataFrameOprEngine.scala | 79 ++++++++++- .../measure/process/engine/DqEngine.scala | 7 +- .../measure/process/engine/DqEngines.scala | 49 ++++++- .../process/engine/SparkSqlEngine.scala | 124 ++++++++++++++---- .../rules/adaptor/DataFrameOprAdaptor.scala | 4 +- .../rules/adaptor/GriffinDslAdaptor.scala | 5 + .../rules/adaptor/SparkSqlAdaptor.scala | 4 +- .../measure/rules/step/ConcreteRuleStep.scala | 10 +- .../measure/rules/step/DfOprStep.scala | 4 +- .../griffin/measure/rules/step/RuleStep.scala | 1 + .../measure/rules/step/SparkSqlStep.scala | 3 +- .../measure/process/BatchProcessTest.scala | 4 +- .../measure/process/JsonParseTest.scala | 2 +- 19 files changed, 319 insertions(+), 96 deletions(-) diff --git a/measure/src/main/scala/org/apache/griffin/measure/persist/HdfsPersist.scala b/measure/src/main/scala/org/apache/griffin/measure/persist/HdfsPersist.scala index 63835620b..23990cfb9 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/persist/HdfsPersist.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/persist/HdfsPersist.scala @@ -21,7 +21,7 @@ package org.apache.griffin.measure.persist import java.util.Date import org.apache.griffin.measure.result._ -import org.apache.griffin.measure.utils.HdfsUtil +import org.apache.griffin.measure.utils.{HdfsUtil, JsonUtil} import org.apache.spark.rdd.RDD import scala.util.Try @@ -41,10 +41,10 @@ case class HdfsPersist(config: Map[String, Any], metricName: String, timeStamp: val StartFile = filePath("_START") val FinishFile = filePath("_FINISH") - val ResultFile = filePath("_RESULT") + val MetricsFile = filePath("_METRICS") - val MissRecFile = filePath("_MISSREC") // optional - val MatchRecFile = filePath("_MATCHREC") // optional +// val MissRecFile = filePath("_MISSREC") // optional +// val MatchRecFile = filePath("_MATCHREC") // optional val LogFile = filePath("_LOG") @@ -224,25 +224,34 @@ case class HdfsPersist(config: Map[String, Any], metricName: String, timeStamp: } } - def persistMetrics(metrics: Seq[String], name: String): Unit = { - val path = filePath(name) +// def persistMetrics(metrics: Seq[String], name: String): Unit = { +// val path = filePath(name) +// try { +// val recordCount = metrics.size +// val count = if (maxPersistLines < 0) recordCount else scala.math.min(maxPersistLines, recordCount) +// if (count > 0) { +// val groupCount = ((count - 1) / maxLinesPerFile + 1).toInt +// if (groupCount <= 1) { +// val recs = metrics.take(count.toInt) +// persistRecords(path, recs) +// } else { +// val groupedRecords = metrics.grouped(groupCount).zipWithIndex +// groupedRecords.take(groupCount).foreach { group => +// val (recs, gid) = group +// val hdfsPath = if (gid == 0) path else withSuffix(path, gid.toString) +// persistRecords(hdfsPath, recs) +// } +// } +// } +// } catch { +// case e: Throwable => error(e.getMessage) +// } +// } + + def persistMetrics(metrics: Map[String, Any]): Unit = { + val json = JsonUtil.toJson(metrics) try { - val recordCount = metrics.size - val count = if (maxPersistLines < 0) recordCount else scala.math.min(maxPersistLines, recordCount) - if (count > 0) { - val groupCount = ((count - 1) / maxLinesPerFile + 1).toInt - if (groupCount <= 1) { - val recs = metrics.take(count.toInt) - persistRecords(path, recs) - } else { - val groupedRecords = metrics.grouped(groupCount).zipWithIndex - groupedRecords.take(groupCount).foreach { group => - val (recs, gid) = group - val hdfsPath = if (gid == 0) path else withSuffix(path, gid.toString) - persistRecords(hdfsPath, recs) - } - } - } + persistRecords(MetricsFile, json :: Nil) } catch { case e: Throwable => error(e.getMessage) } diff --git a/measure/src/main/scala/org/apache/griffin/measure/persist/HttpPersist.scala b/measure/src/main/scala/org/apache/griffin/measure/persist/HttpPersist.scala index 528905421..a15331c1a 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/persist/HttpPersist.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/persist/HttpPersist.scala @@ -33,6 +33,8 @@ case class HttpPersist(config: Map[String, Any], metricName: String, timeStamp: val api = config.getOrElse(Api, "").toString val method = config.getOrElse(Method, "post").toString + val _Value = "value" + def available(): Boolean = { api.nonEmpty } @@ -87,17 +89,23 @@ case class HttpPersist(config: Map[String, Any], metricName: String, timeStamp: def persistRecords(records: RDD[String], name: String): Unit = {} - def persistMetrics(metrics: Seq[String], name: String): Unit = { - val maps = metrics.flatMap { m => - try { - Some(JsonUtil.toAnyMap(m) ++ Map[String, Any](("name" -> metricName), ("tmst" -> timeStamp))) - } catch { - case e: Throwable => None - } - } - maps.foreach { map => - httpResult(map) - } +// def persistMetrics(metrics: Seq[String], name: String): Unit = { +// val maps = metrics.flatMap { m => +// try { +// Some(JsonUtil.toAnyMap(m) ++ Map[String, Any](("name" -> metricName), ("tmst" -> timeStamp))) +// } catch { +// case e: Throwable => None +// } +// } +// maps.foreach { map => +// httpResult(map) +// } +// } + + def persistMetrics(metrics: Map[String, Any]): Unit = { + val head = Map[String, Any](("name" -> metricName), ("tmst" -> timeStamp)) + val result = head + (_Value -> metrics) + httpResult(result) } } diff --git a/measure/src/main/scala/org/apache/griffin/measure/persist/LoggerPersist.scala b/measure/src/main/scala/org/apache/griffin/measure/persist/LoggerPersist.scala index 0bcba6b75..f8d051799 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/persist/LoggerPersist.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/persist/LoggerPersist.scala @@ -128,16 +128,24 @@ case class LoggerPersist(config: Map[String, Any], metricName: String, timeStamp } } - def persistMetrics(metrics: Seq[String], name: String): Unit = { - try { - val recordCount = metrics.size - val count = if (maxLogLines < 0) recordCount else scala.math.min(maxLogLines, recordCount) - if (count > 0) { - val recordsArray = metrics.take(count) - recordsArray.foreach(println) - } - } catch { - case e: Throwable => error(e.getMessage) +// def persistMetrics(metrics: Seq[String], name: String): Unit = { +// try { +// val recordCount = metrics.size +// val count = if (maxLogLines < 0) recordCount else scala.math.min(maxLogLines, recordCount) +// if (count > 0) { +// val recordsArray = metrics.take(count) +// recordsArray.foreach(println) +// } +// } catch { +// case e: Throwable => error(e.getMessage) +// } +// } + + def persistMetrics(metrics: Map[String, Any]): Unit = { + println(s"${metricName} [${timeStamp}] metrics: ") + metrics.foreach { metric => + val (key, value) = metric + println(s"${key}: ${value}") } } diff --git a/measure/src/main/scala/org/apache/griffin/measure/persist/MultiPersists.scala b/measure/src/main/scala/org/apache/griffin/measure/persist/MultiPersists.scala index 095147e27..90550842e 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/persist/MultiPersists.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/persist/MultiPersists.scala @@ -50,6 +50,7 @@ case class MultiPersists(persists: Iterable[Persist]) extends Persist { def log(rt: Long, msg: String): Unit = { persists.foreach(_.log(rt, msg)) } def persistRecords(records: RDD[String], name: String): Unit = { persists.foreach(_.persistRecords(records, name)) } - def persistMetrics(metrics: Seq[String], name: String): Unit = { persists.foreach(_.persistMetrics(metrics, name)) } +// def persistMetrics(metrics: Seq[String], name: String): Unit = { persists.foreach(_.persistMetrics(metrics, name)) } + def persistMetrics(metrics: Map[String, Any]): Unit = { persists.foreach(_.persistMetrics(metrics)) } } diff --git a/measure/src/main/scala/org/apache/griffin/measure/persist/Persist.scala b/measure/src/main/scala/org/apache/griffin/measure/persist/Persist.scala index f76e69c5a..0de763618 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/persist/Persist.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/persist/Persist.scala @@ -43,7 +43,8 @@ trait Persist extends Loggable with Serializable { // def records(recs: Iterable[String], tp: String): Unit def persistRecords(records: RDD[String], name: String): Unit - def persistMetrics(metrics: Seq[String], name: String): Unit +// def persistMetrics(metrics: Seq[String], name: String): Unit + def persistMetrics(metrics: Map[String, Any]): Unit } diff --git a/measure/src/main/scala/org/apache/griffin/measure/process/BatchDqProcess.scala b/measure/src/main/scala/org/apache/griffin/measure/process/BatchDqProcess.scala index 752c727f0..3dbe21142 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/process/BatchDqProcess.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/process/BatchDqProcess.scala @@ -82,7 +82,7 @@ case class BatchDqProcess(allParam: AllParam) extends DqProcess { dqEngines.runRuleSteps(ruleSteps) // persist results - dqEngines.persistResults(ruleSteps, persist) + dqEngines.persistAllResults(ruleSteps, persist) // end time val endTime = new Date().getTime diff --git a/measure/src/main/scala/org/apache/griffin/measure/process/engine/DataFrameOprEngine.scala b/measure/src/main/scala/org/apache/griffin/measure/process/engine/DataFrameOprEngine.scala index 6d3f43344..4d1c71eba 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/process/engine/DataFrameOprEngine.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/process/engine/DataFrameOprEngine.scala @@ -23,7 +23,9 @@ import java.util.Date import org.apache.griffin.measure.config.params.user.DataSourceParam import org.apache.griffin.measure.data.source.{DataSource, DataSourceFactory} import org.apache.griffin.measure.persist.Persist +import org.apache.griffin.measure.rules.dsl._ import org.apache.griffin.measure.rules.step._ +import org.apache.griffin.measure.utils.JsonUtil import org.apache.spark.sql.{DataFrame, SQLContext} import org.apache.spark.streaming.StreamingContext @@ -36,7 +38,7 @@ case class DataFrameOprEngine(sqlContext: SQLContext, @transient ssc: StreamingC def runRuleStep(ruleStep: ConcreteRuleStep): Boolean = { ruleStep match { - case DfOprStep(name, rule, details) => { + case DfOprStep(name, rule, details, _) => { try { rule match { case DataFrameOprs._fromJson => { @@ -59,18 +61,23 @@ case class DataFrameOprEngine(sqlContext: SQLContext, @transient ssc: StreamingC } } - def persistResult(ruleStep: ConcreteRuleStep, persist: Persist): Boolean = { + def persistRecords(ruleStep: ConcreteRuleStep, persist: Persist): Boolean = { val curTime = new Date().getTime ruleStep match { - case DfOprStep(name, _, _) => { + case DfOprStep(name, _, _, RecordPersistType) => { try { - val nonLog = s"[ ${name} ] not persisted" - persist.log(curTime, nonLog) + val pdf = sqlContext.table(s"`${name}`") + val records = pdf.toJSON + + persist.persistRecords(records, name) + + val recordLog = s"[ ${name} ] persist records" + persist.log(curTime, recordLog) true } catch { case e: Throwable => { - error(s"persist result ${ruleStep.name} error: ${e.getMessage}") + error(s"persist result ${name} error: ${e.getMessage}") false } } @@ -79,6 +86,66 @@ case class DataFrameOprEngine(sqlContext: SQLContext, @transient ssc: StreamingC } } + def collectMetrics(ruleStep: ConcreteRuleStep): Map[String, Any] = { + val emptyMap = Map[String, Any]() + ruleStep match { + case DfOprStep(name, _, _, MetricPersistType) => { + try { + val pdf = sqlContext.table(s"`${name}`") + val records = pdf.toJSON.collect() + + if (ruleStep.isArray) { + val arr = records.flatMap { rec => + try { + Some(JsonUtil.toAnyMap(rec)) + } catch { + case e: Throwable => None + } + } + Map[String, Any]((name -> arr)) + } else { + records.headOption match { + case Some(head) => { + try { + JsonUtil.toAnyMap(head) + } catch { + case e: Throwable => emptyMap + } + } + case _ => emptyMap + } + } + } catch { + case e: Throwable => { + error(s"persist result ${name} error: ${e.getMessage}") + emptyMap + } + } + } + case _ => emptyMap + } + } + +// def persistResults(ruleStep: ConcreteRuleStep, persist: Persist): Boolean = { +// val curTime = new Date().getTime +// ruleStep match { +// case DfOprStep(name, _, _) => { +// try { +// val nonLog = s"[ ${name} ] not persisted" +// persist.log(curTime, nonLog) +// +// true +// } catch { +// case e: Throwable => { +// error(s"persist result ${ruleStep.name} error: ${e.getMessage}") +// false +// } +// } +// } +// case _ => false +// } +// } + } object DataFrameOprs { diff --git a/measure/src/main/scala/org/apache/griffin/measure/process/engine/DqEngine.scala b/measure/src/main/scala/org/apache/griffin/measure/process/engine/DqEngine.scala index 493539c4b..cd22d390e 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/process/engine/DqEngine.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/process/engine/DqEngine.scala @@ -22,7 +22,9 @@ import org.apache.griffin.measure.config.params.user.DataSourceParam import org.apache.griffin.measure.data.source.DataSource import org.apache.griffin.measure.log.Loggable import org.apache.griffin.measure.persist.Persist +import org.apache.griffin.measure.rules.dsl._ import org.apache.griffin.measure.rules.step._ +import org.apache.spark.sql.DataFrame trait DqEngine extends Loggable with Serializable { @@ -30,6 +32,9 @@ trait DqEngine extends Loggable with Serializable { def runRuleStep(ruleStep: ConcreteRuleStep): Boolean - def persistResult(ruleStep: ConcreteRuleStep, persist: Persist): Boolean +// def persistResults(ruleSteps: Seq[ConcreteRuleStep], persist: Persist, persistType: PersistType): Boolean + + def persistRecords(ruleStep: ConcreteRuleStep, persist: Persist): Boolean + def collectMetrics(ruleStep: ConcreteRuleStep): Map[String, Any] } diff --git a/measure/src/main/scala/org/apache/griffin/measure/process/engine/DqEngines.scala b/measure/src/main/scala/org/apache/griffin/measure/process/engine/DqEngines.scala index d7ab619c3..cf5345420 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/process/engine/DqEngines.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/process/engine/DqEngines.scala @@ -22,10 +22,13 @@ import org.apache.griffin.measure.config.params.user.DataSourceParam import org.apache.griffin.measure.data.source._ import org.apache.griffin.measure.log.Loggable import org.apache.griffin.measure.persist.Persist +import org.apache.griffin.measure.rules.dsl._ import org.apache.griffin.measure.rules.step._ case class DqEngines(engines: Seq[DqEngine]) extends DqEngine { + val persistOrder: List[PersistType] = List(MetricPersistType, RecordPersistType) + def initDataSources(dataSourceParams: Seq[DataSourceParam]): Unit = { val dataSources = dataSourceParams.flatMap { param => genDataSource(param) @@ -41,9 +44,30 @@ case class DqEngines(engines: Seq[DqEngine]) extends DqEngine { } } - def persistResults(ruleSteps: Seq[ConcreteRuleStep], persist: Persist): Unit = { - ruleSteps.foreach { ruleStep => - persistResult(ruleStep, persist) + def persistAllResults(ruleSteps: Seq[ConcreteRuleStep], persist: Persist): Unit = { + // 1. group by same persist types + val groupedRuleSteps = ruleSteps.groupBy(_.persistType) + + // 2. persist results in order [metric, record] + persistOrder.foreach { prstType => + val steps = groupedRuleSteps.get(prstType) match { + case Some(a) => a + case _ => Nil + } + prstType match { + case MetricPersistType => { + val metrics = steps.foldLeft(Map[String, Any]())(_ ++ collectMetrics(_)) + persist.persistMetrics(metrics) + } + case RecordPersistType => { + steps.foreach { ruleStep => + persistRecords(ruleStep, persist) + } + } + case _ => { + warn(s"${prstType} is not persistable") + } + } } } @@ -63,12 +87,27 @@ case class DqEngines(engines: Seq[DqEngine]) extends DqEngine { ret } - def persistResult(ruleStep: ConcreteRuleStep, persist: Persist): Boolean = { + def persistRecords(ruleStep: ConcreteRuleStep, persist: Persist): Boolean = { val ret = engines.foldLeft(false) { (done, engine) => - done || engine.persistResult(ruleStep, persist) + done || engine.persistRecords(ruleStep, persist) } if (!ret) error(s"persist result warn: no dq engine support ${ruleStep}") ret } + def collectMetrics(ruleStep: ConcreteRuleStep): Map[String, Any] = { + val ret = engines.foldLeft(Map[String, Any]()) { (ret, engine) => + ret ++ engine.collectMetrics(ruleStep) + } + if (ret.isEmpty) error(s"collect metrics warn: no metrics collected for ${ruleStep}") + ret + } + +// def persistResults(ruleSteps: Seq[ConcreteRuleStep], persist: Persist, persistType: PersistType): Boolean = { +// val ret = engines.foldLeft(false) { (done, engine) => +// done || engine.persistResults(ruleSteps, persist, persistType) +// } +// if (!ret) error(s"persist result warn: no dq engine support ${ruleSteps}") +// ret +// } } diff --git a/measure/src/main/scala/org/apache/griffin/measure/process/engine/SparkSqlEngine.scala b/measure/src/main/scala/org/apache/griffin/measure/process/engine/SparkSqlEngine.scala index 2fe1e8a5f..4a90022a0 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/process/engine/SparkSqlEngine.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/process/engine/SparkSqlEngine.scala @@ -23,7 +23,7 @@ import java.util.Date import org.apache.griffin.measure.config.params.user.DataSourceParam import org.apache.griffin.measure.data.source._ import org.apache.griffin.measure.persist.Persist -import org.apache.griffin.measure.rules.dsl.{MetricPersistType, RecordPersistType} +import org.apache.griffin.measure.rules.dsl._ import org.apache.griffin.measure.rules.step._ import org.apache.griffin.measure.utils.JsonUtil import org.apache.spark.sql.{DataFrame, SQLContext} @@ -38,7 +38,7 @@ case class SparkSqlEngine(sqlContext: SQLContext, @transient ssc: StreamingConte def runRuleStep(ruleStep: ConcreteRuleStep): Boolean = { ruleStep match { - case SparkSqlStep(name, rule, _) => { + case SparkSqlStep(name, rule, _, _) => { try { val rdf = sqlContext.sql(rule) rdf.registerTempTable(name) @@ -54,49 +54,121 @@ case class SparkSqlEngine(sqlContext: SQLContext, @transient ssc: StreamingConte } } - def persistResult(ruleStep: ConcreteRuleStep, persist: Persist): Boolean = { + def persistRecords(ruleStep: ConcreteRuleStep, persist: Persist): Boolean = { val curTime = new Date().getTime ruleStep match { - case SparkSqlStep(name, _, persistType) => { + case SparkSqlStep(name, _, _, RecordPersistType) => { try { - persistType match { - case RecordPersistType => { - val pdf = sqlContext.table(s"`${name}`") - val records = pdf.toJSON + val pdf = sqlContext.table(s"`${name}`") + val records = pdf.toJSON - persist.persistRecords(records, name) + persist.persistRecords(records, name) - val recordLog = s"[ ${name} ] persist records" - persist.log(curTime, recordLog) - } - case MetricPersistType => { - val pdf = sqlContext.table(s"`${name}`") - val recordRdd = pdf.toJSON + val recordLog = s"[ ${name} ] persist records" + persist.log(curTime, recordLog) - val metrics = recordRdd.collect - persist.persistMetrics(metrics, name) + true + } catch { + case e: Throwable => { + error(s"persist result ${name} error: ${e.getMessage}") + false + } + } + } + case _ => false + } + } - val metricLog = s"[ ${name} ] persist metric \n${metrics.mkString("\n")}" - persist.log(curTime, metricLog) + def collectMetrics(ruleStep: ConcreteRuleStep): Map[String, Any] = { + val emptyMap = Map[String, Any]() + ruleStep match { + case SparkSqlStep(name, _, _, MetricPersistType) => { + try { + val pdf = sqlContext.table(s"`${name}`") + val records = pdf.toJSON.collect() + + if (ruleStep.isArray) { + val arr = records.flatMap { rec => + try { + Some(JsonUtil.toAnyMap(rec)) + } catch { + case e: Throwable => None + } } - case _ => { - val nonLog = s"[ ${name} ] not persisted" - persist.log(curTime, nonLog) + Map[String, Any]((name -> arr)) + } else { + records.headOption match { + case Some(head) => { + try { + JsonUtil.toAnyMap(head) + } catch { + case e: Throwable => emptyMap + } + } + case _ => emptyMap } } - - true } catch { case e: Throwable => { error(s"persist result ${name} error: ${e.getMessage}") - false + emptyMap } } } - case _ => false + case _ => emptyMap } } +// def persistResults(ruleSteps: Seq[ConcreteRuleStep], persist: Persist, persistType: PersistType): Boolean = { +// val curTime = new Date().getTime +// persistType match { +// case RecordPersistType => { +// ; +// } +// } +// +// +// ruleStep match { +// case SparkSqlStep(name, _, persistType) => { +// try { +// persistType match { +// case RecordPersistType => { +// val pdf = sqlContext.table(s"`${name}`") +// val records = pdf.toJSON +// +// persist.persistRecords(records, name) +// +// val recordLog = s"[ ${name} ] persist records" +// persist.log(curTime, recordLog) +// } +// case MetricPersistType => { +// val pdf = sqlContext.table(s"`${name}`") +// val recordRdd = pdf.toJSON +// +// val metrics = recordRdd.collect +// persist.persistMetrics(metrics, name) +// +// val metricLog = s"[ ${name} ] persist metric \n${metrics.mkString("\n")}" +// persist.log(curTime, metricLog) +// } +// case _ => { +// val nonLog = s"[ ${name} ] not persisted" +// persist.log(curTime, nonLog) +// } +// } +// +// true +// } catch { +// case e: Throwable => { +// error(s"persist result ${name} error: ${e.getMessage}") +// false +// } +// } +// } +// case _ => false +// } +// } + } diff --git a/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/DataFrameOprAdaptor.scala b/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/DataFrameOprAdaptor.scala index 65d648d20..00dfab9af 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/DataFrameOprAdaptor.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/DataFrameOprAdaptor.scala @@ -23,11 +23,11 @@ import org.apache.griffin.measure.rules.step._ case class DataFrameOprAdaptor() extends RuleAdaptor { def genRuleStep(param: Map[String, Any]): Seq[RuleStep] = { - DfOprStep(getName(param), getRule(param), getDetails(param)) :: Nil + DfOprStep(getName(param), getRule(param), getDetails(param), getPersistType(param)) :: Nil } def adaptConcreteRuleStep(ruleStep: RuleStep): Seq[ConcreteRuleStep] = { ruleStep match { - case rs @ DfOprStep(_, _, _) => rs :: Nil + case rs @ DfOprStep(_, _, _, _) => rs :: Nil case _ => Nil } } diff --git a/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/GriffinDslAdaptor.scala b/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/GriffinDslAdaptor.scala index 1d4146dd0..ab68faf3b 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/GriffinDslAdaptor.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/GriffinDslAdaptor.scala @@ -177,6 +177,7 @@ case class GriffinDslAdaptor(dataSourceNames: Seq[String], functionNames: Seq[St val missRecordsStep = SparkSqlStep( missRecordsName, missRecordsSql, + Map[String, Any](), resultPersistType(details, AccuracyInfo._MissRecords, RecordPersistType) ) @@ -189,6 +190,7 @@ case class GriffinDslAdaptor(dataSourceNames: Seq[String], functionNames: Seq[St val missStep = SparkSqlStep( missTableName, missSql, + Map[String, Any](), NonePersistType ) @@ -201,6 +203,7 @@ case class GriffinDslAdaptor(dataSourceNames: Seq[String], functionNames: Seq[St val totalStep = SparkSqlStep( totalTableName, totalSql, + Map[String, Any](), NonePersistType ) @@ -218,6 +221,7 @@ case class GriffinDslAdaptor(dataSourceNames: Seq[String], functionNames: Seq[St val accuracyMetricStep = SparkSqlStep( accuracyMetricName, accuracyMetricSql, + Map[String, Any](), resultPersistType(details, AccuracyInfo._Accuracy, MetricPersistType) ) @@ -246,6 +250,7 @@ case class GriffinDslAdaptor(dataSourceNames: Seq[String], functionNames: Seq[St val profilingStep = SparkSqlStep( profilingMetricName, profilingSql, + Map[String, Any](), resultPersistType(details, ProfilingInfo._Profiling, RecordPersistType) ) diff --git a/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/SparkSqlAdaptor.scala b/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/SparkSqlAdaptor.scala index abe01c845..f4357a429 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/SparkSqlAdaptor.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/SparkSqlAdaptor.scala @@ -25,11 +25,11 @@ import org.apache.griffin.measure.rules.step._ case class SparkSqlAdaptor() extends RuleAdaptor { def genRuleStep(param: Map[String, Any]): Seq[RuleStep] = { - SparkSqlStep(getName(param), getRule(param), getPersistType(param)) :: Nil + SparkSqlStep(getName(param), getRule(param), getDetails(param), getPersistType(param)) :: Nil } def adaptConcreteRuleStep(ruleStep: RuleStep): Seq[ConcreteRuleStep] = { ruleStep match { - case rs @ SparkSqlStep(_, _, _) => rs :: Nil + case rs @ SparkSqlStep(_, _, _, _) => rs :: Nil case _ => Nil } } diff --git a/measure/src/main/scala/org/apache/griffin/measure/rules/step/ConcreteRuleStep.scala b/measure/src/main/scala/org/apache/griffin/measure/rules/step/ConcreteRuleStep.scala index ddb6099cd..162316abd 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rules/step/ConcreteRuleStep.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rules/step/ConcreteRuleStep.scala @@ -18,10 +18,18 @@ under the License. */ package org.apache.griffin.measure.rules.step -import org.apache.griffin.measure.rules.dsl.{DslType, PersistType} +import org.apache.griffin.measure.rules.dsl._ trait ConcreteRuleStep extends RuleStep { val persistType: PersistType + def isArray: Boolean = { + val _IsArray = "is.array" + details.get(_IsArray) match { + case Some(b: Boolean) => b + case _ => false + } + } + } diff --git a/measure/src/main/scala/org/apache/griffin/measure/rules/step/DfOprStep.scala b/measure/src/main/scala/org/apache/griffin/measure/rules/step/DfOprStep.scala index d04a01175..a1162d819 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rules/step/DfOprStep.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rules/step/DfOprStep.scala @@ -20,11 +20,9 @@ package org.apache.griffin.measure.rules.step import org.apache.griffin.measure.rules.dsl._ -case class DfOprStep(name: String, rule: String, details: Map[String, Any] +case class DfOprStep(name: String, rule: String, details: Map[String, Any], persistType: PersistType ) extends ConcreteRuleStep { val dslType: DslType = DfOprType - val persistType: PersistType = NonePersistType - } diff --git a/measure/src/main/scala/org/apache/griffin/measure/rules/step/RuleStep.scala b/measure/src/main/scala/org/apache/griffin/measure/rules/step/RuleStep.scala index e53f07748..9258be067 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rules/step/RuleStep.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rules/step/RuleStep.scala @@ -26,5 +26,6 @@ trait RuleStep extends Serializable { val name: String val rule: String + val details: Map[String, Any] } diff --git a/measure/src/main/scala/org/apache/griffin/measure/rules/step/SparkSqlStep.scala b/measure/src/main/scala/org/apache/griffin/measure/rules/step/SparkSqlStep.scala index 8fcb5f774..602492b60 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rules/step/SparkSqlStep.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rules/step/SparkSqlStep.scala @@ -21,7 +21,8 @@ package org.apache.griffin.measure.rules.step import org.apache.griffin.measure.persist._ import org.apache.griffin.measure.rules.dsl._ -case class SparkSqlStep(name: String, rule: String, persistType: PersistType) extends ConcreteRuleStep { +case class SparkSqlStep(name: String, rule: String, details: Map[String, Any], persistType: PersistType + ) extends ConcreteRuleStep { val dslType: DslType = SparkSqlType diff --git a/measure/src/test/scala/org/apache/griffin/measure/process/BatchProcessTest.scala b/measure/src/test/scala/org/apache/griffin/measure/process/BatchProcessTest.scala index 11fa21d2d..210d1c2b9 100644 --- a/measure/src/test/scala/org/apache/griffin/measure/process/BatchProcessTest.scala +++ b/measure/src/test/scala/org/apache/griffin/measure/process/BatchProcessTest.scala @@ -36,8 +36,8 @@ import scala.util.{Failure, Success, Try} class BatchProcessTest extends FunSuite with Matchers with BeforeAndAfter with Loggable { val envFile = "src/test/resources/env-test.json" - val confFile = "src/test/resources/config-test-profiling.json" -// val confFile = "src/test/resources/config-test2.json" +// val confFile = "src/test/resources/config-test-profiling.json" + val confFile = "src/test/resources/config-test2.json" val envFsType = "local" val userFsType = "local" diff --git a/measure/src/test/scala/org/apache/griffin/measure/process/JsonParseTest.scala b/measure/src/test/scala/org/apache/griffin/measure/process/JsonParseTest.scala index 04414a02a..e68a4c9fa 100644 --- a/measure/src/test/scala/org/apache/griffin/measure/process/JsonParseTest.scala +++ b/measure/src/test/scala/org/apache/griffin/measure/process/JsonParseTest.scala @@ -30,7 +30,7 @@ import org.apache.griffin.measure.process.engine.DataFrameOprs import org.apache.griffin.measure.utils.{HdfsUtil, JsonUtil} import org.apache.hadoop.hive.ql.exec.UDF import org.apache.spark.{SparkConf, SparkContext} -import org.apache.spark.sql.{Column, DataFrame, Row, SQLContext} +import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.expressions.GenericRowWithSchema import org.apache.spark.sql.expressions.UserDefinedAggregateFunction import org.apache.spark.sql.hive.HiveContext From eb9e4828a160b9b9c0f8baa8769f0fcbda0840a5 Mon Sep 17 00:00:00 2001 From: Lionel Liu Date: Fri, 1 Sep 2017 10:17:44 +0800 Subject: [PATCH 060/111] udf and profiling process --- .../measure/persist/LoggerPersist.scala | 12 +- .../measure/process/BatchDqProcess.scala | 4 + .../rules/adaptor/GriffinDslAdaptor.scala | 8 +- .../dsl/analyzer/ProfilingAnalyzer.scala | 9 +- .../rules/dsl/expr/ClauseExpression.scala | 113 ++++++++ .../rules/dsl/parser/BasicParser.scala | 47 +++- .../rules/dsl/parser/GriffinDslParser.scala | 2 +- .../GriffinUdfs.scala} | 16 +- .../test/resources/config-test-profiling.json | 5 +- .../measure/process/BatchProcessTest.scala | 4 +- .../griffin/measure/utils/HdfsUtilTest.scala | 250 +++++++++--------- 11 files changed, 319 insertions(+), 151 deletions(-) create mode 100644 measure/src/main/scala/org/apache/griffin/measure/rules/dsl/expr/ClauseExpression.scala rename measure/src/main/scala/org/apache/griffin/measure/rules/{dsl/expr/Expressions.scala => udf/GriffinUdfs.scala} (72%) diff --git a/measure/src/main/scala/org/apache/griffin/measure/persist/LoggerPersist.scala b/measure/src/main/scala/org/apache/griffin/measure/persist/LoggerPersist.scala index f8d051799..8781c8315 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/persist/LoggerPersist.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/persist/LoggerPersist.scala @@ -21,7 +21,7 @@ package org.apache.griffin.measure.persist import java.util.Date import org.apache.griffin.measure.result._ -import org.apache.griffin.measure.utils.HdfsUtil +import org.apache.griffin.measure.utils.{HdfsUtil, JsonUtil} import org.apache.spark.rdd.RDD // persist result and data to hdfs @@ -143,10 +143,12 @@ case class LoggerPersist(config: Map[String, Any], metricName: String, timeStamp def persistMetrics(metrics: Map[String, Any]): Unit = { println(s"${metricName} [${timeStamp}] metrics: ") - metrics.foreach { metric => - val (key, value) = metric - println(s"${key}: ${value}") - } + val json = JsonUtil.toJson(metrics) + println(json) +// metrics.foreach { metric => +// val (key, value) = metric +// println(s"${key}: ${value}") +// } } diff --git a/measure/src/main/scala/org/apache/griffin/measure/process/BatchDqProcess.scala b/measure/src/main/scala/org/apache/griffin/measure/process/BatchDqProcess.scala index 3dbe21142..0369bdbc4 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/process/BatchDqProcess.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/process/BatchDqProcess.scala @@ -26,6 +26,7 @@ import org.apache.griffin.measure.config.params.user._ import org.apache.griffin.measure.persist.{Persist, PersistFactory} import org.apache.griffin.measure.process.engine.{DqEngineFactory, SparkSqlEngine} import org.apache.griffin.measure.rules.adaptor.RuleAdaptorGroup +import org.apache.griffin.measure.rules.udf.GriffinUdfs import org.apache.griffin.measure.utils.JsonUtil import org.apache.spark.sql.SQLContext import org.apache.spark.sql.hive.HiveContext @@ -53,6 +54,9 @@ case class BatchDqProcess(allParam: AllParam) extends DqProcess { sparkContext.setLogLevel(sparkParam.logLevel) sqlContext = new HiveContext(sparkContext) + // register udf + GriffinUdfs.register(sqlContext) + // init adaptors val dataSourceNames = userParam.dataSources.map(_.name) RuleAdaptorGroup.init(sqlContext, dataSourceNames) diff --git a/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/GriffinDslAdaptor.scala b/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/GriffinDslAdaptor.scala index ab68faf3b..d84a76ca2 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/GriffinDslAdaptor.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/GriffinDslAdaptor.scala @@ -232,7 +232,7 @@ case class GriffinDslAdaptor(dataSourceNames: Seq[String], functionNames: Seq[St case Some(name) => name case _ => dataSourceNames.head } - val analyzer = ProfilingAnalyzer(expr.asInstanceOf[Expressions], sourceName) + val analyzer = ProfilingAnalyzer(expr.asInstanceOf[CombinedClause], sourceName) // 1. select statement val profilingSql = { @@ -244,13 +244,15 @@ case class GriffinDslAdaptor(dataSourceNames: Seq[String], functionNames: Seq[St s"${sel.desc}${alias}" }.mkString(", ") - s"SELECT ${selClause} FROM ${sourceName}" + val tailClause = analyzer.tailsExprs.map(_.desc).mkString(" ") + + s"SELECT ${selClause} FROM ${sourceName} ${tailClause}" } val profilingMetricName = resultName(details, ProfilingInfo._Profiling) val profilingStep = SparkSqlStep( profilingMetricName, profilingSql, - Map[String, Any](), + details, resultPersistType(details, ProfilingInfo._Profiling, RecordPersistType) ) diff --git a/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/analyzer/ProfilingAnalyzer.scala b/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/analyzer/ProfilingAnalyzer.scala index dfa3df4fd..6424eb3d7 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/analyzer/ProfilingAnalyzer.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/analyzer/ProfilingAnalyzer.scala @@ -21,15 +21,18 @@ package org.apache.griffin.measure.rules.dsl.analyzer import org.apache.griffin.measure.rules.dsl.expr._ -case class ProfilingAnalyzer(expr: Expressions, sourceName: String) extends BasicAnalyzer { +case class ProfilingAnalyzer(expr: CombinedClause, sourceName: String) extends BasicAnalyzer { val dataSourceNames = expr.preOrderTraverseDepthFirst(Set[String]())(seqDataSourceNames, combDataSourceNames) val sourceSelectionExprs = { val seq = seqSelectionExprs(sourceName) - expr.preOrderTraverseDepthFirst(Seq[SelectionExpr]())(seq, combSelectionExprs) + expr.selectClause.preOrderTraverseDepthFirst(Seq[SelectionExpr]())(seq, combSelectionExprs) } - val selectionExprs = expr.exprs.map(_.extractSelf) + val selectionExprs = { + expr.selectClause.exprs.map(_.extractSelf) + } + val tailsExprs = expr.tails.map(_.extractSelf) } diff --git a/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/expr/ClauseExpression.scala b/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/expr/ClauseExpression.scala new file mode 100644 index 000000000..0bf365dfd --- /dev/null +++ b/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/expr/ClauseExpression.scala @@ -0,0 +1,113 @@ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ +package org.apache.griffin.measure.rules.dsl.expr + +trait ClauseExpression extends Expr { +} + +case class SelectClause(exprs: Seq[Expr]) extends ClauseExpression { + + addChildren(exprs) + + def desc: String = s"${exprs.map(_.desc).mkString(", ")}" + def coalesceDesc: String = s"${exprs.map(_.desc).mkString(", ")}" + +} + +case class WhereClause(expr: Expr) extends ClauseExpression { + + addChild(expr) + + def desc: String = s"WHERE ${expr.desc}" + def coalesceDesc: String = s"WHERE ${expr.coalesceDesc}" + +} + +case class GroupbyClause(exprs: Seq[Expr], havingClauseOpt: Option[Expr]) extends ClauseExpression { + + addChildren(exprs ++ havingClauseOpt.toSeq) + + def desc: String = { + val gbs = exprs.map(_.desc).mkString(", ") + havingClauseOpt match { +// case Some(having) => s"GROUP BY ${gbs} HAVING ${having.desc}" + case Some(having) => s"GROUP BY ${gbs}" + case _ => s"GROUP BY ${gbs}" + } + } + def coalesceDesc: String = { + val gbs = exprs.map(_.desc).mkString(", ") + havingClauseOpt match { +// case Some(having) => s"GROUP BY ${gbs} HAVING ${having.coalesceDesc}" + case Some(having) => s"GROUP BY ${gbs}" + case _ => s"GROUP BY ${gbs}" + } + } + +} + +case class OrderbyItem(expr: Expr, orderOpt: Option[String]) extends Expr { + addChild(expr) + def desc: String = { + orderOpt match { + case Some(os) => s"${expr.desc} ${os.toUpperCase}" + case _ => s"${expr.desc}" + } + } + def coalesceDesc: String = desc +} + +case class OrderbyClause(items: Seq[OrderbyItem]) extends ClauseExpression { + + addChildren(items.map(_.expr)) + + def desc: String = { + val obs = items.map(_.desc).mkString(", ") + s"ORDER BY ${obs}" + } + def coalesceDesc: String = { + val obs = items.map(_.desc).mkString(", ") + s"ORDER BY ${obs}" + } +} + +case class LimitClause(expr: Expr) extends ClauseExpression { + + addChild(expr) + + def desc: String = s"LIMIT ${expr.desc}" + def coalesceDesc: String = s"LIMIT ${expr.coalesceDesc}" +} + +case class CombinedClause(selectClause: SelectClause, tails: Seq[ClauseExpression] + ) extends ClauseExpression { + + addChildren(selectClause +: tails) + + def desc: String = { + tails.foldLeft(selectClause.desc) { (head, tail) => + s"${head} ${tail.desc}" + } + } + def coalesceDesc: String = { + tails.foldLeft(selectClause.coalesceDesc) { (head, tail) => + s"${head} ${tail.coalesceDesc}" + } + } +} \ No newline at end of file diff --git a/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/parser/BasicParser.scala b/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/parser/BasicParser.scala index ba7d1fcbf..2a022cf72 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/parser/BasicParser.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/parser/BasicParser.scala @@ -79,8 +79,8 @@ trait BasicParser extends JavaTokenParsers with Serializable { */ protected def genNamesParser(names: Seq[String]): Parser[String] = { - names.map { - fn => s"""${fn}""".r: Parser[String] + names.reverse.map { + fn => s"""${fn}""": Parser[String] }.reduce(_ | _) } @@ -119,6 +119,14 @@ trait BasicParser extends JavaTokenParsers with Serializable { val COMMA: Parser[String] = "," val AS: Parser[String] = "(?i)as".r + val WHERE: Parser[String] = "(?i)where".r + val GROUP: Parser[String] = "(?i)group".r + val ORDER: Parser[String] = "(?i)order".r + val BY: Parser[String] = "(?i)by".r + val DESC: Parser[String] = "(?i)desc".r + val ASC: Parser[String] = "(?i)asc".r + val HAVING: Parser[String] = "(?i)having".r + val LIMIT: Parser[String] = "(?i)limit".r } import Operator._ @@ -280,10 +288,39 @@ trait BasicParser extends JavaTokenParsers with Serializable { def argument: Parser[Expr] = expression /** - * -- exprs -- - * = [, ]* + * -- clauses -- + * = [, ]* + * = + * = + * = [ ]? + * = [ ]? + * = [ , ]* + * = */ - def expressions: Parser[Expressions] = rep1sep(expression, COMMA) ^^ { Expressions(_) } + def selectClause: Parser[SelectClause] = rep1sep(expression, COMMA) ^^ { SelectClause(_) } + def whereClause: Parser[WhereClause] = WHERE ~> expression ^^ { WhereClause(_) } + def havingClause: Parser[Expr] = HAVING ~> expression + def groupbyClause: Parser[GroupbyClause] = GROUP ~ BY ~ rep1sep(expression, COMMA) ~ opt(havingClause) ^^ { + case _ ~ _ ~ cols ~ havingOpt => GroupbyClause(cols, havingOpt) + } + def orderbyItem: Parser[OrderbyItem] = expression ~ opt(DESC | ASC) ^^ { + case expr ~ orderOpt => OrderbyItem(expr, orderOpt) + } + def orderbyClause: Parser[OrderbyClause] = ORDER ~> BY ~> rep1sep(orderbyItem, COMMA) ^^ { OrderbyClause(_) } + def limitClause: Parser[LimitClause] = LIMIT ~> expression ^^ { LimitClause(_) } + + /** + * -- combined clauses -- + * = [ ]+ [ ]+ [ ]+ [ ]+ + */ + + def combinedClause: Parser[CombinedClause] = selectClause ~ opt(whereClause) ~ + opt(groupbyClause) ~ opt(orderbyClause) ~ opt(limitClause) ^^ { + case sel ~ whereOpt ~ groupbyOpt ~ orderbyOpt ~ limitOpt => { + val tails = Seq(whereOpt, groupbyOpt, orderbyOpt, limitOpt).flatMap(opt => opt) + CombinedClause(sel, tails) + } + } } diff --git a/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/parser/GriffinDslParser.scala b/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/parser/GriffinDslParser.scala index 987986249..b3959caf3 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/parser/GriffinDslParser.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/parser/GriffinDslParser.scala @@ -27,7 +27,7 @@ case class GriffinDslParser(dataSourceNames: Seq[String], functionNames: Seq[Str def parseRule(rule: String, dqType: DqType): ParseResult[Expr] = { val rootExpr = dqType match { case AccuracyType => logicalExpression - case ProfilingType => expressions + case ProfilingType => combinedClause case _ => expression } parseAll(rootExpr, rule) diff --git a/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/expr/Expressions.scala b/measure/src/main/scala/org/apache/griffin/measure/rules/udf/GriffinUdfs.scala similarity index 72% rename from measure/src/main/scala/org/apache/griffin/measure/rules/dsl/expr/Expressions.scala rename to measure/src/main/scala/org/apache/griffin/measure/rules/udf/GriffinUdfs.scala index 81e35dfa4..a4bded366 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/expr/Expressions.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rules/udf/GriffinUdfs.scala @@ -16,12 +16,18 @@ KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License. */ -package org.apache.griffin.measure.rules.dsl.expr +package org.apache.griffin.measure.rules.udf -case class Expressions(exprs: Seq[Expr]) extends Expr { +import org.apache.spark.sql.SQLContext - addChildren(exprs) +object GriffinUdfs { + + def register(sqlContext: SQLContext): Unit = { + sqlContext.udf.register("index_of", indexOf) + } + + private val indexOf = (arr: Seq[String], v: String) => { + arr.indexOf(v) + } - def desc: String = s"${exprs.map(_.desc).mkString(", ")}" - def coalesceDesc: String = s"${exprs.map(_.coalesceDesc).mkString(", ")}" } \ No newline at end of file diff --git a/measure/src/test/resources/config-test-profiling.json b/measure/src/test/resources/config-test-profiling.json index 201a3d4dd..67095f1eb 100644 --- a/measure/src/test/resources/config-test-profiling.json +++ b/measure/src/test/resources/config-test-profiling.json @@ -23,13 +23,14 @@ { "dsl.type": "griffin-dsl", "dq.type": "profiling", - "rule": "source.user_id.max(), source.first_name.count() as cnt", + "rule": "count(source.user_id), source.first_name group by source.first_name order by count(source.user_id) desc, source.first_name asc limit 3", "details": { "source": "source", "profiling": { "name": "prof", "persist.type": "metric" - } + }, + "is.array": true } } ] diff --git a/measure/src/test/scala/org/apache/griffin/measure/process/BatchProcessTest.scala b/measure/src/test/scala/org/apache/griffin/measure/process/BatchProcessTest.scala index 210d1c2b9..11fa21d2d 100644 --- a/measure/src/test/scala/org/apache/griffin/measure/process/BatchProcessTest.scala +++ b/measure/src/test/scala/org/apache/griffin/measure/process/BatchProcessTest.scala @@ -36,8 +36,8 @@ import scala.util.{Failure, Success, Try} class BatchProcessTest extends FunSuite with Matchers with BeforeAndAfter with Loggable { val envFile = "src/test/resources/env-test.json" -// val confFile = "src/test/resources/config-test-profiling.json" - val confFile = "src/test/resources/config-test2.json" + val confFile = "src/test/resources/config-test-profiling.json" +// val confFile = "src/test/resources/config-test2.json" val envFsType = "local" val userFsType = "local" diff --git a/measure/src/test/scala/org/apache/griffin/measure/utils/HdfsUtilTest.scala b/measure/src/test/scala/org/apache/griffin/measure/utils/HdfsUtilTest.scala index e360c1dfe..90db32af1 100644 --- a/measure/src/test/scala/org/apache/griffin/measure/utils/HdfsUtilTest.scala +++ b/measure/src/test/scala/org/apache/griffin/measure/utils/HdfsUtilTest.scala @@ -1,132 +1,132 @@ -///* -//Licensed to the Apache Software Foundation (ASF) under one -//or more contributor license agreements. See the NOTICE file -//distributed with this work for additional information -//regarding copyright ownership. The ASF licenses this file -//to you under the Apache License, Version 2.0 (the -//"License"); you may not use this file except in compliance -//with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -//Unless required by applicable law or agreed to in writing, -//software distributed under the License is distributed on an -//"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -//KIND, either express or implied. See the License for the -//specific language governing permissions and limitations -//under the License. -//*/ -//package org.apache.griffin.measure.utils -// -//import java.io.{BufferedReader, FileReader, InputStreamReader} -// -//import org.apache.hadoop.conf.Configuration -//import org.apache.hadoop.fs.{FileSystem, Path} -//import org.apache.spark.sql.SQLContext -//import org.apache.spark.sql.hive.HiveContext -//import org.apache.spark.{SparkConf, SparkContext} -//import org.junit.runner.RunWith -//import org.scalatest.junit.JUnitRunner -//import org.scalatest.{BeforeAndAfter, FunSuite, Matchers} -// -//@RunWith(classOf[JUnitRunner]) -//class HdfsUtilTest extends FunSuite with Matchers with BeforeAndAfter { -// -// private val seprator = "/" -// -// private val conf1 = new Configuration() -// conf1.addResource(new Path("file:///apache/hadoop/etc/hadoop/core-site.xml")) -// conf1.addResource(new Path("file:///apache/hadoop/etc/hadoop/hdfs-site.xml")) -// private val dfs1 = FileSystem.get(conf1) -// -// private val conf2 = new Configuration() -// conf2.addResource(new Path("file:///Users/lliu13/test/hadoop/core-site.xml")) -// conf2.addResource(new Path("file:///Users/lliu13/test/hadoop/hdfs-site.xml")) -// private val dfs2 = FileSystem.get(conf2) -// -// val conf = new SparkConf().setAppName("test_hdfs").setMaster("local[*]") -// val sparkContext = new SparkContext(conf) -// sparkContext.setLogLevel("WARN") -// val sqlContext = new HiveContext(sparkContext) -// -// def listSubPaths(dfs: FileSystem, dirPath: String, subType: String, fullPath: Boolean = false): Iterable[String] = { -// val path = new Path(dirPath) +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ +package org.apache.griffin.measure.utils + +import java.io.{BufferedReader, FileReader, InputStreamReader} + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.{FileSystem, Path} +import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.hive.HiveContext +import org.apache.spark.{SparkConf, SparkContext} +import org.junit.runner.RunWith +import org.scalatest.junit.JUnitRunner +import org.scalatest.{BeforeAndAfter, FunSuite, Matchers} + +@RunWith(classOf[JUnitRunner]) +class HdfsUtilTest extends FunSuite with Matchers with BeforeAndAfter { + + private val seprator = "/" + + private val conf1 = new Configuration() + conf1.addResource(new Path("file:///apache/hadoop/etc/hadoop/core-site.xml")) + conf1.addResource(new Path("file:///apache/hadoop/etc/hadoop/hdfs-site.xml")) + private val dfs1 = FileSystem.get(conf1) + + private val conf2 = new Configuration() + conf2.addResource(new Path("file:///Users/lliu13/test/hadoop/core-site.xml")) + conf2.addResource(new Path("file:///Users/lliu13/test/hadoop/hdfs-site.xml")) + private val dfs2 = FileSystem.get(conf2) + + val conf = new SparkConf().setAppName("test_hdfs").setMaster("local[*]") + val sparkContext = new SparkContext(conf) + sparkContext.setLogLevel("WARN") + val sqlContext = new HiveContext(sparkContext) + + def listSubPaths(dfs: FileSystem, dirPath: String, subType: String, fullPath: Boolean = false): Iterable[String] = { + val path = new Path(dirPath) + try { + val fileStatusArray = dfs.listStatus(path) + fileStatusArray.filter { fileStatus => + subType match { + case "dir" => fileStatus.isDirectory + case "file" => fileStatus.isFile + case _ => true + } + }.map { fileStatus => + val fname = fileStatus.getPath.getName + if (fullPath) getHdfsFilePath(dirPath, fname) else fname + } + } catch { + case e: Throwable => { + println(s"list path files error: ${e.getMessage}") + Nil + } + } + } + + def getHdfsFilePath(parentPath: String, fileName: String): String = { + if (parentPath.endsWith(seprator)) parentPath + fileName else parentPath + seprator + fileName + } + + test ("test multiple hdfs") { + val list1 = listSubPaths(dfs1, "/", "dir", false) + println(list1) + + val list2 = listSubPaths(dfs2, "/", "dir", false) + println(list2) + + val path1 = "/depth/discovery_file_sample.txt" + val istream1 = dfs1.open(new Path(path1)) + val reader1 = new BufferedReader(new InputStreamReader(istream1)) + val seq1 = scala.collection.mutable.MutableList[String]() + try { + var line = reader1.readLine() + while (line != null) { + val arr = line.split("\u0007") + seq1 ++= arr + line = reader1.readLine() + } + } finally { + reader1.close() + istream1.close() + } + +// val scanner = new java.util.Scanner(istream1,"UTF-8").useDelimiter("\u0007") +// val theString = if (scanner.hasNext()) scanner.next() else "" +// println(theString) +// scanner.close() + + println(seq1.size) + println(seq1.take(10)) + seq1.take(10).foreach(println) + +// val path2 = "/griffin/json/env.json" +// val istream2 = dfs2.open(new Path(path2)) +// val reader2 = new BufferedReader(new InputStreamReader(istream2)) +// val seq2 = scala.collection.mutable.MutableList[String]() // try { -// val fileStatusArray = dfs.listStatus(path) -// fileStatusArray.filter { fileStatus => -// subType match { -// case "dir" => fileStatus.isDirectory -// case "file" => fileStatus.isFile -// case _ => true -// } -// }.map { fileStatus => -// val fname = fileStatus.getPath.getName -// if (fullPath) getHdfsFilePath(dirPath, fname) else fname +// var line = reader2.readLine() +// while (line != null) { +// line = reader2.readLine() +// seq2 += line // } // } catch { // case e: Throwable => { -// println(s"list path files error: ${e.getMessage}") -// Nil -// } -// } -// } -// -// def getHdfsFilePath(parentPath: String, fileName: String): String = { -// if (parentPath.endsWith(seprator)) parentPath + fileName else parentPath + seprator + fileName -// } -// -// test ("test multiple hdfs") { -// val list1 = listSubPaths(dfs1, "/", "dir", false) -// println(list1) -// -// val list2 = listSubPaths(dfs2, "/", "dir", false) -// println(list2) -// -// val path1 = "/depth/depth.bz2" -// val istream1 = dfs1.open(new Path(path1)) -// val reader1 = new BufferedReader(new InputStreamReader(istream1)) -// val seq1 = scala.collection.mutable.MutableList[String]() -// try { -// var line = reader1.readLine() -// while (line != null) { -// val arr = line.split("\u0007") -// seq1 ++= arr -// line = reader1.readLine() +// println("error in reading") // } // } finally { -// reader1.close() -// istream1.close() +// reader2.close() +// istream2.close() // } -// -//// val scanner = new java.util.Scanner(istream1,"UTF-8").useDelimiter("\u0007") -//// val theString = if (scanner.hasNext()) scanner.next() else "" -//// println(theString) -//// scanner.close() -// -// println(seq1.size) -// println(seq1.take(10)) -// seq1.take(10).foreach(println) -// -//// val path2 = "/griffin/json/env.json" -//// val istream2 = dfs2.open(new Path(path2)) -//// val reader2 = new BufferedReader(new InputStreamReader(istream2)) -//// val seq2 = scala.collection.mutable.MutableList[String]() -//// try { -//// var line = reader2.readLine() -//// while (line != null) { -//// line = reader2.readLine() -//// seq2 += line -//// } -//// } catch { -//// case e: Throwable => { -//// println("error in reading") -//// } -//// } finally { -//// reader2.close() -//// istream2.close() -//// } -//// println(seq2.size) -//// println(seq2.take(10)) -// } -// -//} +// println(seq2.size) +// println(seq2.take(10)) + } + +} From 86ddb56934926dc90347a6940913b142e77ddfb9 Mon Sep 17 00:00:00 2001 From: Lionel Liu Date: Fri, 1 Sep 2017 14:44:40 +0800 Subject: [PATCH 061/111] init 1 --- .../resources/{config-test2.json => config-test-accuracy.json} | 0 .../org/apache/griffin/measure/process/BatchProcessTest.scala | 2 +- 2 files changed, 1 insertion(+), 1 deletion(-) rename measure/src/test/resources/{config-test2.json => config-test-accuracy.json} (100%) diff --git a/measure/src/test/resources/config-test2.json b/measure/src/test/resources/config-test-accuracy.json similarity index 100% rename from measure/src/test/resources/config-test2.json rename to measure/src/test/resources/config-test-accuracy.json diff --git a/measure/src/test/scala/org/apache/griffin/measure/process/BatchProcessTest.scala b/measure/src/test/scala/org/apache/griffin/measure/process/BatchProcessTest.scala index 11fa21d2d..94998da97 100644 --- a/measure/src/test/scala/org/apache/griffin/measure/process/BatchProcessTest.scala +++ b/measure/src/test/scala/org/apache/griffin/measure/process/BatchProcessTest.scala @@ -37,7 +37,7 @@ class BatchProcessTest extends FunSuite with Matchers with BeforeAndAfter with L val envFile = "src/test/resources/env-test.json" val confFile = "src/test/resources/config-test-profiling.json" -// val confFile = "src/test/resources/config-test2.json" +// val confFile = "src/test/resources/config-test-accuracy.json" val envFsType = "local" val userFsType = "local" From 4372ba08dd8f44711d03ce593a69c3553bcb99eb Mon Sep 17 00:00:00 2001 From: Lionel Liu Date: Fri, 1 Sep 2017 15:49:46 +0800 Subject: [PATCH 062/111] group.metric --- .../process/engine/DataFrameOprEngine.scala | 2 +- .../measure/process/engine/SparkSqlEngine.scala | 2 +- .../measure/rules/step/ConcreteRuleStep.scala | 6 +++--- .../src/test/resources/config-test-accuracy.json | 16 ++++++++-------- .../test/resources/config-test-profiling.json | 2 +- 5 files changed, 14 insertions(+), 14 deletions(-) diff --git a/measure/src/main/scala/org/apache/griffin/measure/process/engine/DataFrameOprEngine.scala b/measure/src/main/scala/org/apache/griffin/measure/process/engine/DataFrameOprEngine.scala index 4d1c71eba..a80af7216 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/process/engine/DataFrameOprEngine.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/process/engine/DataFrameOprEngine.scala @@ -94,7 +94,7 @@ case class DataFrameOprEngine(sqlContext: SQLContext, @transient ssc: StreamingC val pdf = sqlContext.table(s"`${name}`") val records = pdf.toJSON.collect() - if (ruleStep.isArray) { + if (ruleStep.isGroupMetric) { val arr = records.flatMap { rec => try { Some(JsonUtil.toAnyMap(rec)) diff --git a/measure/src/main/scala/org/apache/griffin/measure/process/engine/SparkSqlEngine.scala b/measure/src/main/scala/org/apache/griffin/measure/process/engine/SparkSqlEngine.scala index 4a90022a0..ac75b794d 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/process/engine/SparkSqlEngine.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/process/engine/SparkSqlEngine.scala @@ -87,7 +87,7 @@ case class SparkSqlEngine(sqlContext: SQLContext, @transient ssc: StreamingConte val pdf = sqlContext.table(s"`${name}`") val records = pdf.toJSON.collect() - if (ruleStep.isArray) { + if (ruleStep.isGroupMetric) { val arr = records.flatMap { rec => try { Some(JsonUtil.toAnyMap(rec)) diff --git a/measure/src/main/scala/org/apache/griffin/measure/rules/step/ConcreteRuleStep.scala b/measure/src/main/scala/org/apache/griffin/measure/rules/step/ConcreteRuleStep.scala index 162316abd..d01f0290f 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rules/step/ConcreteRuleStep.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rules/step/ConcreteRuleStep.scala @@ -24,9 +24,9 @@ trait ConcreteRuleStep extends RuleStep { val persistType: PersistType - def isArray: Boolean = { - val _IsArray = "is.array" - details.get(_IsArray) match { + def isGroupMetric: Boolean = { + val _GroupMetric = "group.metric" + details.get(_GroupMetric) match { case Some(b: Boolean) => b case _ => false } diff --git a/measure/src/test/resources/config-test-accuracy.json b/measure/src/test/resources/config-test-accuracy.json index 4940a72ff..ecbdaaa8f 100644 --- a/measure/src/test/resources/config-test-accuracy.json +++ b/measure/src/test/resources/config-test-accuracy.json @@ -5,7 +5,7 @@ "data.sources": [ { - "name": "source", + "name": "src", "connectors": [ { "type": "avro", @@ -16,7 +16,7 @@ } ] }, { - "name": "target", + "name": "tgt", "connectors": [ { "type": "avro", @@ -34,10 +34,10 @@ { "dsl.type": "griffin-dsl", "dq.type": "accuracy", - "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", + "rule": "src.user_id = tgt.user_id AND upper(src.first_name) = upper(tgt.first_name) AND src.last_name = tgt.last_name AND src.address = tgt.address AND src.email = tgt.email AND src.phone = tgt.phone AND src.post_code = tgt.post_code", "details": { - "source": "source", - "target": "target", + "source": "src", + "target": "tgt", "miss.records": { "name": "miss.records", "persist.type": "record" @@ -46,9 +46,9 @@ "name": "accu", "persist.type": "metric" }, - "miss": "miss", - "total": "total", - "matched": "matched" + "miss": "miss_count", + "total": "total_count", + "matched": "matched_count" } } ] diff --git a/measure/src/test/resources/config-test-profiling.json b/measure/src/test/resources/config-test-profiling.json index 67095f1eb..731c4dddc 100644 --- a/measure/src/test/resources/config-test-profiling.json +++ b/measure/src/test/resources/config-test-profiling.json @@ -30,7 +30,7 @@ "name": "prof", "persist.type": "metric" }, - "is.array": true + "group.metric": true } } ] From 75e51dcc24e397cb451e7d913c6a16b4dce80d14 Mon Sep 17 00:00:00 2001 From: Lionel Liu Date: Thu, 7 Sep 2017 11:19:54 +0800 Subject: [PATCH 063/111] running --- .../config/params/user/DataCacheParam.scala | 61 +++-- .../params/user/DataConnectorParam.scala | 6 +- .../config/params/user/DataSourceParam.scala | 3 +- .../data/connector/DataConnector.scala | 8 +- .../data/connector/DataConnectorFactory.scala | 141 ++++++----- .../AvroBatchDataConnector.scala} | 7 +- .../BatchDataConnector.scala} | 8 +- .../HiveBatchDataConnector.scala} | 7 +- .../KafkaCacheDirectDataConnector.scala | 0 .../StreamingCacheDirectDataConnector.scala | 0 .../KafkaStreamingDataConnector.scala | 131 ++++++----- .../streaming/StreamingDataConnector.scala | 77 +++--- .../measure/data/source/DataCacheable.scala | 76 ++++++ .../measure/data/source/DataSource.scala | 47 +++- .../measure/data/source/DataSourceCache.scala | 219 ++++++++++++++++++ .../data/source/DataSourceFactory.scala | 49 +++- .../measure/process/BatchDqProcess.scala | 12 +- .../measure/process/StreamingDqProcess.scala | 117 +++++++++- .../process/engine/DataFrameOprEngine.scala | 6 +- .../measure/process/engine/DqEngine.scala | 2 +- .../measure/process/engine/DqEngines.scala | 24 +- .../process/engine/SparkSqlEngine.scala | 6 +- .../config-test-accuracy-streaming.json | 102 ++++++++ 23 files changed, 880 insertions(+), 229 deletions(-) rename measure/src/main/scala/org/apache/griffin/measure/data/connector/{direct/AvroDirectDataConnector.scala => batch/AvroBatchDataConnector.scala} (92%) rename measure/src/main/scala/org/apache/griffin/measure/data/connector/{direct/DirectDataConnector.scala => batch/BatchDataConnector.scala} (83%) rename measure/src/main/scala/org/apache/griffin/measure/data/connector/{direct/HiveDirectDataConnector.scala => batch/HiveBatchDataConnector.scala} (94%) rename measure/src/main/scala/org/apache/griffin/measure/data/connector/{direct => batch}/KafkaCacheDirectDataConnector.scala (100%) rename measure/src/main/scala/org/apache/griffin/measure/data/connector/{direct => batch}/StreamingCacheDirectDataConnector.scala (100%) create mode 100644 measure/src/main/scala/org/apache/griffin/measure/data/source/DataCacheable.scala create mode 100644 measure/src/main/scala/org/apache/griffin/measure/data/source/DataSourceCache.scala create mode 100644 measure/src/test/resources/config-test-accuracy-streaming.json diff --git a/measure/src/main/scala/org/apache/griffin/measure/config/params/user/DataCacheParam.scala b/measure/src/main/scala/org/apache/griffin/measure/config/params/user/DataCacheParam.scala index 9c607558a..f94fdb61b 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/config/params/user/DataCacheParam.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/config/params/user/DataCacheParam.scala @@ -1,31 +1,30 @@ -/* -Licensed to the Apache Software Foundation (ASF) under one -or more contributor license agreements. See the NOTICE file -distributed with this work for additional information -regarding copyright ownership. The ASF licenses this file -to you under the Apache License, Version 2.0 (the -"License"); you may not use this file except in compliance -with the License. You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, -software distributed under the License is distributed on an -"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -KIND, either express or implied. See the License for the -specific language governing permissions and limitations -under the License. -*/ -package org.apache.griffin.measure.config.params.user - -import com.fasterxml.jackson.annotation.{JsonInclude, JsonProperty} -import com.fasterxml.jackson.annotation.JsonInclude.Include -import org.apache.griffin.measure.config.params.Param - -@JsonInclude(Include.NON_NULL) -case class DataCacheParam( @JsonProperty("type") cacheType: String, - @JsonProperty("config") config: Map[String, Any], - @JsonProperty("time.range") timeRange: List[String] - ) extends Param { - -} +///* +//Licensed to the Apache Software Foundation (ASF) under one +//or more contributor license agreements. See the NOTICE file +//distributed with this work for additional information +//regarding copyright ownership. The ASF licenses this file +//to you under the Apache License, Version 2.0 (the +//"License"); you may not use this file except in compliance +//with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +//Unless required by applicable law or agreed to in writing, +//software distributed under the License is distributed on an +//"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +//KIND, either express or implied. See the License for the +//specific language governing permissions and limitations +//under the License. +//*/ +//package org.apache.griffin.measure.config.params.user +// +//import com.fasterxml.jackson.annotation.{JsonInclude, JsonProperty} +//import com.fasterxml.jackson.annotation.JsonInclude.Include +//import org.apache.griffin.measure.config.params.Param +// +//@JsonInclude(Include.NON_NULL) +//case class DataCacheParam( @JsonProperty("config") config: Map[String, Any], +// @JsonProperty("time.range") timeRange: List[String] +// ) extends Param { +// +//} diff --git a/measure/src/main/scala/org/apache/griffin/measure/config/params/user/DataConnectorParam.scala b/measure/src/main/scala/org/apache/griffin/measure/config/params/user/DataConnectorParam.scala index dbc2e0baf..37877b9fd 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/config/params/user/DataConnectorParam.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/config/params/user/DataConnectorParam.scala @@ -26,12 +26,8 @@ import org.apache.griffin.measure.config.params.Param case class DataConnectorParam( @JsonProperty("type") conType: String, @JsonProperty("version") version: String, @JsonProperty("config") config: Map[String, Any], - @JsonProperty("cache") cache: DataCacheParam, - @JsonProperty("match.once") matchOnce: Boolean + @JsonProperty("pre.proc") preProc: List[RuleParam] ) extends Param { - def getMatchOnce(): Boolean = { - if (matchOnce == null) false else matchOnce - } } \ No newline at end of file diff --git a/measure/src/main/scala/org/apache/griffin/measure/config/params/user/DataSourceParam.scala b/measure/src/main/scala/org/apache/griffin/measure/config/params/user/DataSourceParam.scala index 48ba81a97..b63823401 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/config/params/user/DataSourceParam.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/config/params/user/DataSourceParam.scala @@ -24,7 +24,8 @@ import org.apache.griffin.measure.config.params.Param @JsonInclude(Include.NON_NULL) case class DataSourceParam( @JsonProperty("name") name: String, - @JsonProperty("connectors") connectors: List[DataConnectorParam] + @JsonProperty("connectors") connectors: List[DataConnectorParam], + @JsonProperty("cache") cache: Map[String, Any] ) extends Param { } diff --git a/measure/src/main/scala/org/apache/griffin/measure/data/connector/DataConnector.scala b/measure/src/main/scala/org/apache/griffin/measure/data/connector/DataConnector.scala index a766b42ca..d8a51a7c8 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/data/connector/DataConnector.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/data/connector/DataConnector.scala @@ -27,4 +27,10 @@ trait DataConnector extends Loggable with Serializable { // def available(): Boolean -} + def init(): Unit + + def data(): Option[DataFrame] + +// def preProcess(dfOpt: Option[DataFrame]): Option[DataFrame] + +} \ No newline at end of file diff --git a/measure/src/main/scala/org/apache/griffin/measure/data/connector/DataConnectorFactory.scala b/measure/src/main/scala/org/apache/griffin/measure/data/connector/DataConnectorFactory.scala index 7ee42a473..2983f9c93 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/data/connector/DataConnectorFactory.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/data/connector/DataConnectorFactory.scala @@ -20,8 +20,13 @@ package org.apache.griffin.measure.data.connector import kafka.serializer.StringDecoder import org.apache.griffin.measure.config.params.user._ +import org.apache.griffin.measure.data.connector.streaming.{KafkaStreamingDataConnector, StreamingDataConnector} +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.DataFrame + +import scala.util.Success //import org.apache.griffin.measure.data.connector.cache._ -import org.apache.griffin.measure.data.connector.direct._ +import org.apache.griffin.measure.data.connector.batch._ //import org.apache.griffin.measure.data.connector.streaming._ import org.apache.griffin.measure.rule.RuleExprs import org.apache.spark.sql.SQLContext @@ -44,39 +49,37 @@ object DataConnectorFactory { def getDirectDataConnector(sqlContext: SQLContext, ssc: StreamingContext, dataConnectorParam: DataConnectorParam - ): Try[DirectDataConnector] = { + ): Try[DataConnector] = { val conType = dataConnectorParam.conType val version = dataConnectorParam.version val config = dataConnectorParam.config Try { conType match { - case HiveRegex() => HiveDirectDataConnector(sqlContext, config) - case AvroRegex() => AvroDirectDataConnector(sqlContext, config) -// case KafkaRegex() => { + case HiveRegex() => HiveBatchDataConnector(sqlContext, config) + case AvroRegex() => AvroBatchDataConnector(sqlContext, config) + case KafkaRegex() => { // val ksdcTry = getStreamingDataConnector(ssc, dataConnectorParam) // val cdcTry = getCacheDataConnector(sqlContext, dataConnectorParam.cache) // KafkaCacheDirectDataConnector(ksdcTry, cdcTry, dataConnectorParam) -// } + getStreamingDataConnector(sqlContext, ssc, dataConnectorParam) + } case _ => throw new Exception("connector creation error!") } } } -// private def getStreamingDataConnector(ssc: StreamingContext, -// dataConnectorParam: DataConnectorParam -// ): Try[StreamingDataConnector] = { -// val conType = dataConnectorParam.conType -// val version = dataConnectorParam.version -// val config = dataConnectorParam.config -// Try { -// conType match { -// case KafkaRegex() => { -// genKafkaDataConnector(ssc, config) -// } -// case _ => throw new Exception("streaming connector creation error!") -// } -// } -// } + private def getStreamingDataConnector(sqlContext: SQLContext, + ssc: StreamingContext, + dataConnectorParam: DataConnectorParam + ): StreamingDataConnector = { + val conType = dataConnectorParam.conType + val version = dataConnectorParam.version + val config = dataConnectorParam.config + conType match { + case KafkaRegex() => genKafkaDataConnector(sqlContext, ssc, config) + case _ => throw new Exception("streaming connector creation error!") + } + } // // private def getCacheDataConnector(sqlContext: SQLContext, // dataCacheParam: DataCacheParam @@ -94,37 +97,69 @@ object DataConnectorFactory { // } // } // -// private def genKafkaDataConnector(ssc: StreamingContext, config: Map[String, Any]) = { -// val KeyType = "key.type" -// val ValueType = "value.type" -// val keyType = config.getOrElse(KeyType, "java.lang.String").toString -// val valueType = config.getOrElse(ValueType, "java.lang.String").toString -// (getClassTag(keyType), getClassTag(valueType)) match { -// case (ClassTag(k: Class[String]), ClassTag(v: Class[String])) => { -// if (ssc == null) throw new Exception("streaming context is null! ") -// new KafkaStreamingDataConnector(ssc, config) { -// type K = String -// type KD = StringDecoder -// type V = String -// type VD = StringDecoder -// def createDStream(topicSet: Set[String]): InputDStream[(K, V)] = { -// KafkaUtils.createDirectStream[K, V, KD, VD](ssc, kafkaConfig, topicSet) -// } -// } -// } -// case _ => { -// throw new Exception("not supported type kafka data connector") -// } -// } -// } -// -// private def getClassTag(tp: String): ClassTag[_] = { -// try { -// val clazz = Class.forName(tp) -// ClassTag(clazz) -// } catch { -// case e: Throwable => throw e -// } -// } + private def genKafkaDataConnector(sqlContext: SQLContext, + ssc: StreamingContext, + config: Map[String, Any] + ) = { + val KeyType = "key.type" + val ValueType = "value.type" + val keyType = config.getOrElse(KeyType, "java.lang.String").toString + val valueType = config.getOrElse(ValueType, "java.lang.String").toString + (getClassTag(keyType), getClassTag(valueType)) match { + case (ClassTag(k: Class[String]), ClassTag(v: Class[String])) => { + if (ssc == null) throw new Exception("streaming context is null! ") + new KafkaStreamingDataConnector(sqlContext, ssc, config) { + type K = String + type KD = StringDecoder + type V = String + type VD = StringDecoder + case class Value(value: K, _tmst: Long) {} + def createDStream(topicSet: Set[String]): InputDStream[(K, V)] = { + KafkaUtils.createDirectStream[K, V, KD, VD](ssc, kafkaConfig, topicSet) + } + def transform(rdd: RDD[(K, V)], ms: Long): Option[DataFrame] = { + val rdd1 = rdd.map(d => Value(d._2, ms)) + try { + Some(sqlContext.createDataFrame(rdd1, classOf[Value])) + } catch { + case e: Throwable => { + error(s"streaming data transform fails") + None + } + } + } + } + } + case _ => { + throw new Exception("not supported type kafka data connector") + } + } + } + + private def getClassTag(tp: String): ClassTag[_] = { + try { + val clazz = Class.forName(tp) + ClassTag(clazz) + } catch { + case e: Throwable => throw e + } + } + + def filterBatchDataConnectors(connectors: Seq[DataConnector]): Seq[BatchDataConnector] = { + connectors.flatMap { dc => + dc match { + case mdc: BatchDataConnector => Some(mdc) + case _ => None + } + } + } + def filterStreamingDataConnectors(connectors: Seq[DataConnector]): Seq[StreamingDataConnector] = { + connectors.flatMap { dc => + dc match { + case mdc: StreamingDataConnector => Some(mdc) + case _ => None + } + } + } } diff --git a/measure/src/main/scala/org/apache/griffin/measure/data/connector/direct/AvroDirectDataConnector.scala b/measure/src/main/scala/org/apache/griffin/measure/data/connector/batch/AvroBatchDataConnector.scala similarity index 92% rename from measure/src/main/scala/org/apache/griffin/measure/data/connector/direct/AvroDirectDataConnector.scala rename to measure/src/main/scala/org/apache/griffin/measure/data/connector/batch/AvroBatchDataConnector.scala index 476401d32..bf4923a0c 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/data/connector/direct/AvroDirectDataConnector.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/data/connector/batch/AvroBatchDataConnector.scala @@ -16,8 +16,9 @@ KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License. */ -package org.apache.griffin.measure.data.connector.direct +package org.apache.griffin.measure.data.connector.batch +import org.apache.griffin.measure.data.connector._ import org.apache.griffin.measure.result._ import org.apache.griffin.measure.rule.{ExprValueUtil, RuleExprs} import org.apache.griffin.measure.utils.HdfsUtil @@ -27,8 +28,8 @@ import org.apache.spark.sql.{DataFrame, SQLContext} import scala.util.Try // data connector for avro file -case class AvroDirectDataConnector(sqlContext: SQLContext, config: Map[String, Any] - ) extends DirectDataConnector { +case class AvroBatchDataConnector(sqlContext: SQLContext, config: Map[String, Any] + ) extends BatchDataConnector { val FilePath = "file.path" val FileName = "file.name" diff --git a/measure/src/main/scala/org/apache/griffin/measure/data/connector/direct/DirectDataConnector.scala b/measure/src/main/scala/org/apache/griffin/measure/data/connector/batch/BatchDataConnector.scala similarity index 83% rename from measure/src/main/scala/org/apache/griffin/measure/data/connector/direct/DirectDataConnector.scala rename to measure/src/main/scala/org/apache/griffin/measure/data/connector/batch/BatchDataConnector.scala index 93e168e78..4d138ab7d 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/data/connector/direct/DirectDataConnector.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/data/connector/batch/BatchDataConnector.scala @@ -16,9 +16,9 @@ KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License. */ -package org.apache.griffin.measure.data.connector.direct +package org.apache.griffin.measure.data.connector.batch -import org.apache.griffin.measure.data.connector.DataConnector +import org.apache.griffin.measure.data.connector._ //import org.apache.griffin.measure.data.connector.cache.DataUpdatable import org.apache.spark.sql.DataFrame import org.apache.spark.sql.types.StructType @@ -26,10 +26,10 @@ import org.apache.spark.sql.types.StructType import scala.util.{Failure, Success, Try} -trait DirectDataConnector extends DataConnector { +trait BatchDataConnector extends DataConnector { // def metaData(): Option[StructType] - def data(): Option[DataFrame] + def init(): Unit = {} } diff --git a/measure/src/main/scala/org/apache/griffin/measure/data/connector/direct/HiveDirectDataConnector.scala b/measure/src/main/scala/org/apache/griffin/measure/data/connector/batch/HiveBatchDataConnector.scala similarity index 94% rename from measure/src/main/scala/org/apache/griffin/measure/data/connector/direct/HiveDirectDataConnector.scala rename to measure/src/main/scala/org/apache/griffin/measure/data/connector/batch/HiveBatchDataConnector.scala index a2fed3ef6..2abca8220 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/data/connector/direct/HiveDirectDataConnector.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/data/connector/batch/HiveBatchDataConnector.scala @@ -16,8 +16,9 @@ KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License. */ -package org.apache.griffin.measure.data.connector.direct +package org.apache.griffin.measure.data.connector.batch +import org.apache.griffin.measure.data.connector._ import org.apache.griffin.measure.result._ import org.apache.griffin.measure.rule.{ExprValueUtil, RuleExprs} import org.apache.spark.rdd.RDD @@ -27,8 +28,8 @@ import org.apache.spark.sql.{DataFrame, SQLContext} import scala.util.{Success, Try} // data connector for hive -case class HiveDirectDataConnector(sqlContext: SQLContext, config: Map[String, Any] - ) extends DirectDataConnector { +case class HiveBatchDataConnector(sqlContext: SQLContext, config: Map[String, Any] + ) extends BatchDataConnector { if (!sqlContext.isInstanceOf[HiveContext]) { throw new Exception("hive context not prepared!") diff --git a/measure/src/main/scala/org/apache/griffin/measure/data/connector/direct/KafkaCacheDirectDataConnector.scala b/measure/src/main/scala/org/apache/griffin/measure/data/connector/batch/KafkaCacheDirectDataConnector.scala similarity index 100% rename from measure/src/main/scala/org/apache/griffin/measure/data/connector/direct/KafkaCacheDirectDataConnector.scala rename to measure/src/main/scala/org/apache/griffin/measure/data/connector/batch/KafkaCacheDirectDataConnector.scala diff --git a/measure/src/main/scala/org/apache/griffin/measure/data/connector/direct/StreamingCacheDirectDataConnector.scala b/measure/src/main/scala/org/apache/griffin/measure/data/connector/batch/StreamingCacheDirectDataConnector.scala similarity index 100% rename from measure/src/main/scala/org/apache/griffin/measure/data/connector/direct/StreamingCacheDirectDataConnector.scala rename to measure/src/main/scala/org/apache/griffin/measure/data/connector/batch/StreamingCacheDirectDataConnector.scala diff --git a/measure/src/main/scala/org/apache/griffin/measure/data/connector/streaming/KafkaStreamingDataConnector.scala b/measure/src/main/scala/org/apache/griffin/measure/data/connector/streaming/KafkaStreamingDataConnector.scala index 0abb826ef..cabdc34ba 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/data/connector/streaming/KafkaStreamingDataConnector.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/data/connector/streaming/KafkaStreamingDataConnector.scala @@ -1,58 +1,73 @@ -///* -//Licensed to the Apache Software Foundation (ASF) under one -//or more contributor license agreements. See the NOTICE file -//distributed with this work for additional information -//regarding copyright ownership. The ASF licenses this file -//to you under the Apache License, Version 2.0 (the -//"License"); you may not use this file except in compliance -//with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -//Unless required by applicable law or agreed to in writing, -//software distributed under the License is distributed on an -//"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -//KIND, either express or implied. See the License for the -//specific language governing permissions and limitations -//under the License. -//*/ -//package org.apache.griffin.measure.data.connector.streaming -// -//import kafka.serializer.Decoder -//import org.apache.griffin.measure.data.connector.cache.{CacheDataConnector, DataCacheable} -//import org.apache.griffin.measure.result.{DataInfo, TimeStampInfo} -//import org.apache.griffin.measure.rule.{ExprValueUtil, RuleExprs} -//import org.apache.spark.rdd.RDD -//import org.apache.spark.streaming.StreamingContext -//import org.apache.spark.streaming.dstream.InputDStream -// -//import scala.util.{Failure, Success, Try} -// -//abstract class KafkaStreamingDataConnector(@transient ssc: StreamingContext, -// config: Map[String, Any] -// ) extends StreamingDataConnector { -// type KD <: Decoder[K] -// type VD <: Decoder[V] -// -// val KafkaConfig = "kafka.config" -// val Topics = "topics" -// -// val kafkaConfig = config.get(KafkaConfig) match { -// case Some(map: Map[String, Any]) => map.mapValues(_.toString).map(identity) -// case _ => Map[String, String]() -// } -// val topics = config.getOrElse(Topics, "").toString -// -// def available(): Boolean = { -// true -// } -// -// def init(): Unit = {} -// -// def stream(): Try[InputDStream[(K, V)]] = Try { -// val topicSet = topics.split(",").toSet -// createDStream(topicSet) -// } -// -// protected def createDStream(topicSet: Set[String]): InputDStream[(K, V)] -//} \ No newline at end of file +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ +package org.apache.griffin.measure.data.connector.streaming + +import kafka.serializer.Decoder +import org.apache.griffin.measure.result.{DataInfo, TimeStampInfo} +import org.apache.griffin.measure.rule.{ExprValueUtil, RuleExprs} +import org.apache.griffin.measure.utils.JsonUtil +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.SQLContext +import org.apache.spark.streaming.StreamingContext +import org.apache.spark.streaming.dstream.InputDStream + +import scala.util.{Failure, Success, Try} + +abstract class KafkaStreamingDataConnector(sqlContext: SQLContext, + @transient ssc: StreamingContext, + config: Map[String, Any] + ) extends StreamingDataConnector { + type KD <: Decoder[K] + type VD <: Decoder[V] + + val KafkaConfig = "kafka.config" + val Topics = "topics" + + val kafkaConfig = config.get(KafkaConfig) match { + case Some(map: Map[String, Any]) => map.mapValues(_.toString).map(identity) + case _ => Map[String, String]() + } + val topics = config.getOrElse(Topics, "").toString + + def available(): Boolean = { + true + } + + def init(): Unit = { + val ds = stream match { + case Success(dstream) => dstream + case Failure(ex) => throw ex + } + ds.foreachRDD((rdd, time) => { + val ms = time.milliseconds + + val dfOpt = transform(rdd, ms) + + // save data frame + dataSourceCacheOpt.foreach(_.saveData(dfOpt, ms)) + }) + } + + def stream(): Try[InputDStream[(K, V)]] = Try { + val topicSet = topics.split(",").toSet + createDStream(topicSet) + } + + protected def createDStream(topicSet: Set[String]): InputDStream[(K, V)] +} \ No newline at end of file diff --git a/measure/src/main/scala/org/apache/griffin/measure/data/connector/streaming/StreamingDataConnector.scala b/measure/src/main/scala/org/apache/griffin/measure/data/connector/streaming/StreamingDataConnector.scala index 3267ec77b..ef5799695 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/data/connector/streaming/StreamingDataConnector.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/data/connector/streaming/StreamingDataConnector.scala @@ -1,34 +1,43 @@ -///* -//Licensed to the Apache Software Foundation (ASF) under one -//or more contributor license agreements. See the NOTICE file -//distributed with this work for additional information -//regarding copyright ownership. The ASF licenses this file -//to you under the Apache License, Version 2.0 (the -//"License"); you may not use this file except in compliance -//with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -//Unless required by applicable law or agreed to in writing, -//software distributed under the License is distributed on an -//"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -//KIND, either express or implied. See the License for the -//specific language governing permissions and limitations -//under the License. -//*/ -//package org.apache.griffin.measure.data.connector.streaming -// -//import org.apache.griffin.measure.data.connector.DataConnector -//import org.apache.spark.streaming.dstream.InputDStream -// -//import scala.util.Try -// -// -//trait StreamingDataConnector extends DataConnector { -// -// type K -// type V -// -// def stream(): Try[InputDStream[(K, V)]] -// -//} +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ +package org.apache.griffin.measure.data.connector.streaming + +import org.apache.griffin.measure.data.connector._ +import org.apache.griffin.measure.data.source.DataSourceCache +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.DataFrame +import org.apache.spark.streaming.dstream.InputDStream + +import scala.util.Try + + +trait StreamingDataConnector extends DataConnector { + + type K + type V + + protected def stream(): Try[InputDStream[(K, V)]] + + def transform(rdd: RDD[(K, V)], ms: Long): Option[DataFrame] + + def data(): Option[DataFrame] = None + + var dataSourceCacheOpt: Option[DataSourceCache] = None + +} diff --git a/measure/src/main/scala/org/apache/griffin/measure/data/source/DataCacheable.scala b/measure/src/main/scala/org/apache/griffin/measure/data/source/DataCacheable.scala new file mode 100644 index 000000000..3c9106a0c --- /dev/null +++ b/measure/src/main/scala/org/apache/griffin/measure/data/source/DataCacheable.scala @@ -0,0 +1,76 @@ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ +package org.apache.griffin.measure.data.source + +import java.util.concurrent.atomic.AtomicLong + +import org.apache.griffin.measure.cache.info.{InfoCacheInstance, TimeInfoCache} + +trait DataCacheable { + + val cacheInfoPath: String + val readyTimeInterval: Long + val readyTimeDelay: Long + + def selfCacheInfoPath = s"${TimeInfoCache.infoPath}/${cacheInfoPath}" + + def selfCacheTime = TimeInfoCache.cacheTime(selfCacheInfoPath) + def selfLastProcTime = TimeInfoCache.lastProcTime(selfCacheInfoPath) + def selfReadyTime = TimeInfoCache.readyTime(selfCacheInfoPath) + def selfCleanTime = TimeInfoCache.cleanTime(selfCacheInfoPath) + + protected def submitCacheTime(ms: Long): Unit = { + val map = Map[String, String]((selfCacheTime -> ms.toString)) + InfoCacheInstance.cacheInfo(map) + } + + protected def submitReadyTime(ms: Long): Unit = { + val curReadyTime = ms - readyTimeDelay + if (curReadyTime % readyTimeInterval == 0) { + val map = Map[String, String]((selfReadyTime -> curReadyTime.toString)) + InfoCacheInstance.cacheInfo(map) + } + } + + protected def submitLastProcTime(ms: Long): Unit = { + val map = Map[String, String]((selfLastProcTime -> ms.toString)) + InfoCacheInstance.cacheInfo(map) + } + + protected def submitCleanTime(ms: Long): Unit = { + val cleanTime = genCleanTime(ms) + val map = Map[String, String]((selfCleanTime -> cleanTime.toString)) + InfoCacheInstance.cacheInfo(map) + } + + protected def genCleanTime(ms: Long): Long = ms + + protected def readCleanTime(): Option[Long] = { + val key = selfCleanTime + val keys = key :: Nil + InfoCacheInstance.readInfo(keys).get(key).flatMap { v => + try { + Some(v.toLong) + } catch { + case _ => None + } + } + } + +} \ No newline at end of file diff --git a/measure/src/main/scala/org/apache/griffin/measure/data/source/DataSource.scala b/measure/src/main/scala/org/apache/griffin/measure/data/source/DataSource.scala index 0b31a8ed4..ea071a9bf 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/data/source/DataSource.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/data/source/DataSource.scala @@ -1,12 +1,44 @@ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ package org.apache.griffin.measure.data.source -import org.apache.griffin.measure.data.connector.direct._ +import org.apache.griffin.measure.data.connector._ +import org.apache.griffin.measure.data.connector.batch._ +import org.apache.griffin.measure.data.connector.streaming._ import org.apache.griffin.measure.log.Loggable import org.apache.spark.sql.{DataFrame, SQLContext} -case class DataSource(name: String, dataConnectors: Seq[DirectDataConnector]) extends Loggable with Serializable { +case class DataSource(name: String, + dataConnectors: Seq[DataConnector], + dataSourceCacheOpt: Option[DataSourceCache] + ) extends Loggable with Serializable { + + val batchDataConnectors = DataConnectorFactory.filterBatchDataConnectors(dataConnectors) + val streamingDataConnectors = DataConnectorFactory.filterStreamingDataConnectors(dataConnectors) + streamingDataConnectors.foreach(_.dataSourceCacheOpt = dataSourceCacheOpt) def init(): Unit = { + dataSourceCacheOpt.foreach(_.init) + dataConnectors.foreach(_.init) + } + + def loadData(): Unit = { data match { case Some(df) => { df.registerTempTable(name) @@ -18,9 +50,18 @@ case class DataSource(name: String, dataConnectors: Seq[DirectDataConnector]) ex } private def data(): Option[DataFrame] = { - dataConnectors.flatMap { dc => + val batchDataFrameOpt = batchDataConnectors.flatMap { dc => dc.data }.reduceOption(_ unionAll _) + + val cacheDataFrameOpt = dataSourceCacheOpt.flatMap(_.readData()) + + (batchDataFrameOpt, cacheDataFrameOpt) match { + case (Some(bdf), Some(cdf)) => Some(bdf unionAll cdf) + case (Some(bdf), _) => Some(bdf) + case (_, Some(cdf)) => Some(cdf) + case _ => None + } } } diff --git a/measure/src/main/scala/org/apache/griffin/measure/data/source/DataSourceCache.scala b/measure/src/main/scala/org/apache/griffin/measure/data/source/DataSourceCache.scala new file mode 100644 index 000000000..ad1f37720 --- /dev/null +++ b/measure/src/main/scala/org/apache/griffin/measure/data/source/DataSourceCache.scala @@ -0,0 +1,219 @@ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ +package org.apache.griffin.measure.data.source + +import java.util.concurrent.TimeUnit + +import org.apache.griffin.measure.cache.info.{InfoCacheInstance, TimeInfoCache} +import org.apache.griffin.measure.data.connector.streaming.StreamingDataConnector +import org.apache.griffin.measure.data.connector._ +import org.apache.griffin.measure.log.Loggable +import org.apache.griffin.measure.utils.{HdfsFileDumpUtil, HdfsUtil, TimeUtil} +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.{DataFrame, SQLContext} + +import scala.util.{Failure, Success} + +case class DataSourceCache(sqlContext: SQLContext, param: Map[String, Any], + metricName: String, index: Int + ) extends DataCacheable with Loggable with Serializable { + + val name = "" + + val _FilePath = "file.path" + val _InfoPath = "info.path" + val _ReadyTimeInterval = "ready.time.interval" + val _ReadyTimeDelay = "ready.time.delay" + val _TimeRange = "time.range" + + val defFilePath = s"hdfs:///griffin/cache/${metricName}/${index}" + val defInfoPath = s"${index}" + + val filePath: String = param.getOrElse(_FilePath, defFilePath).toString + val cacheInfoPath: String = param.getOrElse(_InfoPath, defInfoPath).toString + val readyTimeInterval: Long = TimeUtil.milliseconds(param.getOrElse(_ReadyTimeInterval, "1m").toString).getOrElse(60000L) + val readyTimeDelay: Long = TimeUtil.milliseconds(param.getOrElse(_ReadyTimeDelay, "1m").toString).getOrElse(60000L) + val deltaTimeRange: (Long, Long) = { + def negative(n: Long): Long = if (n <= 0) n else 0 + param.get(_TimeRange) match { + case Some(seq: Seq[String]) => { + val nseq = seq.flatMap(TimeUtil.milliseconds(_)) + val ns = negative(nseq.headOption.getOrElse(0)) + val ne = negative(nseq.tail.headOption.getOrElse(0)) + (ns, ne) + } + case _ => (0, 0) + } + } + + val rowSepLiteral = "\n" + val partitionUnits: List[String] = List("hour", "min") + + val newCacheLock = InfoCacheInstance.genLock(s"${cacheInfoPath}.new") + val oldCacheLock = InfoCacheInstance.genLock(s"${cacheInfoPath}.old") + + def init(): Unit = { + ; + } + + def saveData(dfOpt: Option[DataFrame], ms: Long): Unit = { + dfOpt match { + case Some(df) => { + val newCacheLocked = newCacheLock.lock(-1, TimeUnit.SECONDS) + if (newCacheLocked) { + try { + val ptns = getPartition(ms) + val ptnsPath = genPartitionHdfsPath(ptns) + val dirPath = s"${filePath}/${ptnsPath}" + val dataFileName = s"${ms}" + val dataFilePath = HdfsUtil.getHdfsFilePath(dirPath, dataFileName) + + // transform data + val dataRdd: RDD[String] = df.toJSON + + // save data + val dumped = if (!dataRdd.isEmpty) { + HdfsFileDumpUtil.dump(dataFilePath, dataRdd, rowSepLiteral) + } else false + + // submit ms + submitCacheTime(ms) + submitReadyTime(ms) + } catch { + case e: Throwable => error(s"save data error: ${e.getMessage}") + } finally { + newCacheLock.unlock() + } + } + } + case _ => { + info(s"no data frame to save") + } + } + + } + + def readData(): Option[DataFrame] = { + val timeRange = TimeInfoCache.getTimeRange + submitLastProcTime(timeRange._2) + + val reviseTimeRange = (timeRange._1 + deltaTimeRange._1, timeRange._2 + deltaTimeRange._2) + submitCleanTime(reviseTimeRange._1) + + // read directly through partition info + val partitionRanges = getPartitionRange(reviseTimeRange._1, reviseTimeRange._2) + println(s"read time ranges: ${reviseTimeRange}") + println(s"read partition ranges: ${partitionRanges}") + + // list partition paths + val partitionPaths = listPathsBetweenRanges(filePath :: Nil, partitionRanges) + + if (partitionPaths.isEmpty) { + None + } else { +// val filePaths = partitionPaths.mkString(",") +// val rdd: RDD[String] = sqlContext.sparkContext.textFile(filePaths) +// +// // decode data +// rdd.flatMap { row => +// decode(row) +// } + val df = sqlContext.read.text(partitionPaths: _*) + Some(df) + } + } + + + private def getPartition(ms: Long): List[Long] = { + partitionUnits.map { unit => + TimeUtil.timeToUnit(ms, unit) + } + } + private def getPartitionRange(ms1: Long, ms2: Long): List[(Long, Long)] = { + partitionUnits.map { unit => + val t1 = TimeUtil.timeToUnit(ms1, unit) + val t2 = TimeUtil.timeToUnit(ms2, unit) + (t1, t2) + } + } + private def genPartitionHdfsPath(partition: List[Long]): String = { + partition.map(prtn => s"${prtn}").mkString("/") + } + private def str2Long(str: String): Option[Long] = { + try { + Some(str.toLong) + } catch { + case e: Throwable => None + } + } + + + // here the range means [min, max], but the best range should be (min, max] + private def listPathsBetweenRanges(paths: List[String], + partitionRanges: List[(Long, Long)] + ): List[String] = { + partitionRanges match { + case Nil => paths + case head :: tail => { + val (lb, ub) = head + val curPaths = paths.flatMap { path => + val names = HdfsUtil.listSubPaths(path, "dir").toList + names.filter { name => + str2Long(name) match { + case Some(t) => (t >= lb) && (t <= ub) + case _ => false + } + }.map(HdfsUtil.getHdfsFilePath(path, _)) + } + listPathsBetweenRanges(curPaths, tail) + } + } + } + private def listPathsEarlierThanBounds(paths: List[String], bounds: List[Long] + ): List[String] = { + bounds match { + case Nil => paths + case head :: tail => { + val earlierPaths = paths.flatMap { path => + val names = HdfsUtil.listSubPaths(path, "dir").toList + names.filter { name => + str2Long(name) match { + case Some(t) => (t < head) + case _ => false + } + }.map(HdfsUtil.getHdfsFilePath(path, _)) + } + val equalPaths = paths.flatMap { path => + val names = HdfsUtil.listSubPaths(path, "dir").toList + names.filter { name => + str2Long(name) match { + case Some(t) => (t == head) + case _ => false + } + }.map(HdfsUtil.getHdfsFilePath(path, _)) + } + + tail match { + case Nil => earlierPaths + case _ => earlierPaths ::: listPathsEarlierThanBounds(equalPaths, tail) + } + } + } + } +} diff --git a/measure/src/main/scala/org/apache/griffin/measure/data/source/DataSourceFactory.scala b/measure/src/main/scala/org/apache/griffin/measure/data/source/DataSourceFactory.scala index c737a8435..328279faf 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/data/source/DataSourceFactory.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/data/source/DataSourceFactory.scala @@ -18,26 +18,61 @@ under the License. */ package org.apache.griffin.measure.data.source -import org.apache.griffin.measure.config.params.user.DataSourceParam -import org.apache.griffin.measure.data.connector.DataConnectorFactory +import org.apache.griffin.measure.config.params.user._ +import org.apache.griffin.measure.data.connector.batch.BatchDataConnector +import org.apache.griffin.measure.data.connector.streaming.StreamingDataConnector +import org.apache.griffin.measure.data.connector.{DataConnector, DataConnectorFactory} +import org.apache.griffin.measure.log.Loggable import org.apache.spark.sql.SQLContext import org.apache.spark.streaming.StreamingContext -import scala.util.Success +import scala.util.{Success, Try} -object DataSourceFactory { +object DataSourceFactory extends Loggable { - def genDataSource(sqlContext: SQLContext, ssc: StreamingContext, dataSourceParam: DataSourceParam - ): Option[DataSource] = { + val HiveRegex = """^(?i)hive$""".r + val TextRegex = """^(?i)text$""".r + val AvroRegex = """^(?i)avro$""".r + + def genDataSources(sqlContext: SQLContext, ssc: StreamingContext, + dataSourceParams: Seq[DataSourceParam], metricName: String): Seq[DataSource] = { + dataSourceParams.zipWithIndex.flatMap { pair => + val (param, index) = pair + genDataSource(sqlContext, ssc, param, metricName, index) + } + } + + private def genDataSource(sqlContext: SQLContext, ssc: StreamingContext, + dataSourceParam: DataSourceParam, + metricName: String, index: Int + ): Option[DataSource] = { val name = dataSourceParam.name val connectorParams = dataSourceParam.connectors + val cacheParam = dataSourceParam.cache val dataConnectors = connectorParams.flatMap { connectorParam => DataConnectorFactory.getDirectDataConnector(sqlContext, ssc, connectorParam) match { case Success(connector) => Some(connector) case _ => None } } - Some(DataSource(name, dataConnectors)) + val dataSourceCacheOpt = genDataSourceCache(sqlContext, cacheParam, metricName, index) + + Some(DataSource(name, dataConnectors, dataSourceCacheOpt)) + } + + private def genDataSourceCache(sqlContext: SQLContext, param: Map[String, Any], + metricName: String, index: Int + ) = { + if (param != null) { + try { + Some(DataSourceCache(sqlContext, param, metricName, index)) + } catch { + case e: Throwable => { + error(s"generate data source cache fails") + None + } + } + } else None } } diff --git a/measure/src/main/scala/org/apache/griffin/measure/process/BatchDqProcess.scala b/measure/src/main/scala/org/apache/griffin/measure/process/BatchDqProcess.scala index 0369bdbc4..e500a892a 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/process/BatchDqProcess.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/process/BatchDqProcess.scala @@ -23,6 +23,7 @@ import java.util.Date import org.apache.griffin.measure.config.params._ import org.apache.griffin.measure.config.params.env._ import org.apache.griffin.measure.config.params.user._ +import org.apache.griffin.measure.data.source.DataSourceFactory import org.apache.griffin.measure.persist.{Persist, PersistFactory} import org.apache.griffin.measure.process.engine.{DqEngineFactory, SparkSqlEngine} import org.apache.griffin.measure.rules.adaptor.RuleAdaptorGroup @@ -73,14 +74,17 @@ case class BatchDqProcess(allParam: AllParam) extends DqProcess { val applicationId = sparkContext.applicationId persist.start(applicationId) - // generate rule steps - val ruleSteps = RuleAdaptorGroup.genConcreteRuleSteps(userParam.evaluateRuleParam) - // get dq engines val dqEngines = DqEngineFactory.genDqEngines(sqlContext, null) + // generate data sources + val dataSources = DataSourceFactory.genDataSources(sqlContext, null, userParam.dataSources, metricName) + // init data sources - dqEngines.initDataSources(userParam.dataSources) + dqEngines.loadData(dataSources) + + // generate rule steps + val ruleSteps = RuleAdaptorGroup.genConcreteRuleSteps(userParam.evaluateRuleParam) // run rules dqEngines.runRuleSteps(ruleSteps) diff --git a/measure/src/main/scala/org/apache/griffin/measure/process/StreamingDqProcess.scala b/measure/src/main/scala/org/apache/griffin/measure/process/StreamingDqProcess.scala index 08c4b7aab..107a574bd 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/process/StreamingDqProcess.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/process/StreamingDqProcess.scala @@ -18,9 +18,23 @@ under the License. */ package org.apache.griffin.measure.process +import java.util.Date + +import org.apache.griffin.measure.algo.streaming.TimingProcess +import org.apache.griffin.measure.cache.info.InfoCacheInstance import org.apache.griffin.measure.config.params._ import org.apache.griffin.measure.config.params.env._ import org.apache.griffin.measure.config.params.user._ +import org.apache.griffin.measure.data.source.DataSourceFactory +import org.apache.griffin.measure.persist.{Persist, PersistFactory} +import org.apache.griffin.measure.process.engine.DqEngineFactory +import org.apache.griffin.measure.rules.adaptor.RuleAdaptorGroup +import org.apache.griffin.measure.rules.udf.GriffinUdfs +import org.apache.griffin.measure.utils.TimeUtil +import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.hive.HiveContext +import org.apache.spark.streaming.{Milliseconds, StreamingContext} +import org.apache.spark.{SparkConf, SparkContext} import scala.util.Try @@ -29,14 +43,111 @@ case class StreamingDqProcess(allParam: AllParam) extends DqProcess { val envParam: EnvParam = allParam.envParam val userParam: UserParam = allParam.userParam + val metricName = userParam.name + val sparkParam = envParam.sparkParam + + var sparkContext: SparkContext = _ + var sqlContext: SQLContext = _ + def retriable: Boolean = true - def init: Try[_] = Try {} + def init: Try[_] = Try { + val conf = new SparkConf().setAppName(metricName) + conf.setAll(sparkParam.config) + sparkContext = new SparkContext(conf) + sparkContext.setLogLevel(sparkParam.logLevel) + sqlContext = new HiveContext(sparkContext) + + // init info cache instance + InfoCacheInstance.initInstance(envParam.infoCacheParams, metricName) + InfoCacheInstance.init + + // register udf + GriffinUdfs.register(sqlContext) + + // init adaptors + val dataSourceNames = userParam.dataSources.map(_.name) + RuleAdaptorGroup.init(sqlContext, dataSourceNames) + } def run: Try[_] = Try { - ; + val ssc = StreamingContext.getOrCreate(sparkParam.cpDir, () => { + try { + createStreamingContext + } catch { + case e: Throwable => { + error(s"create streaming context error: ${e.getMessage}") + throw e + } + } + }) + + // start time + val startTime = new Date().getTime() + + // get persists to persist measure result + val persist: Persist = PersistFactory(envParam.persistParams, metricName).getPersists(startTime) + + // persist start id + val applicationId = sparkContext.applicationId + persist.start(applicationId) + + // get dq engines + val dqEngines = DqEngineFactory.genDqEngines(sqlContext, ssc) + + // generate data sources + val dataSources = DataSourceFactory.genDataSources(sqlContext, null, userParam.dataSources, metricName) + + // init data sources + dqEngines.loadData(dataSources) + + // generate rule steps + val ruleSteps = RuleAdaptorGroup.genConcreteRuleSteps(userParam.evaluateRuleParam) + + // run rules + dqEngines.runRuleSteps(ruleSteps) + + // persist results + dqEngines.persistAllResults(ruleSteps, persist) + + // end time + val endTime = new Date().getTime + persist.log(endTime, s"process using time: ${endTime - startTime} ms") + +// val processInterval = TimeUtil.milliseconds(sparkParam.processInterval) match { +// case Some(interval) => interval +// case _ => throw new Exception("invalid batch interval") +// } +// val process = TimingProcess(processInterval, streamingAccuracyProcess) +// process.startup() + + ssc.start() + ssc.awaitTermination() + ssc.stop(stopSparkContext=true, stopGracefully=true) + + // finish + persist.finish() + +// process.shutdown() } - def end: Try[_] = Try {} + def end: Try[_] = Try { + sparkContext.stop + + InfoCacheInstance.close + } + + def createStreamingContext: StreamingContext = { + val batchInterval = TimeUtil.milliseconds(sparkParam.batchInterval) match { + case Some(interval) => Milliseconds(interval) + case _ => throw new Exception("invalid batch interval") + } + val ssc = new StreamingContext(sparkContext, batchInterval) + ssc.checkpoint(sparkParam.cpDir) + + + + ssc + } } diff --git a/measure/src/main/scala/org/apache/griffin/measure/process/engine/DataFrameOprEngine.scala b/measure/src/main/scala/org/apache/griffin/measure/process/engine/DataFrameOprEngine.scala index a80af7216..a2e9fd5a4 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/process/engine/DataFrameOprEngine.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/process/engine/DataFrameOprEngine.scala @@ -32,9 +32,9 @@ import org.apache.spark.streaming.StreamingContext case class DataFrameOprEngine(sqlContext: SQLContext, @transient ssc: StreamingContext ) extends DqEngine { - def genDataSource(dataSourceParam: DataSourceParam): Option[DataSource] = { - DataSourceFactory.genDataSource(sqlContext, ssc, dataSourceParam) - } +// def genDataSource(dataSourceParam: DataSourceParam): Option[DirectDataSource] = { +// DataSourceFactory.genDataSource(sqlContext, ssc, dataSourceParam) +// } def runRuleStep(ruleStep: ConcreteRuleStep): Boolean = { ruleStep match { diff --git a/measure/src/main/scala/org/apache/griffin/measure/process/engine/DqEngine.scala b/measure/src/main/scala/org/apache/griffin/measure/process/engine/DqEngine.scala index cd22d390e..c2507def8 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/process/engine/DqEngine.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/process/engine/DqEngine.scala @@ -28,7 +28,7 @@ import org.apache.spark.sql.DataFrame trait DqEngine extends Loggable with Serializable { - def genDataSource(dataSourceParam: DataSourceParam): Option[DataSource] +// def genDataSource(dataSourceParam: DataSourceParam): Option[DirectDataSource] def runRuleStep(ruleStep: ConcreteRuleStep): Boolean diff --git a/measure/src/main/scala/org/apache/griffin/measure/process/engine/DqEngines.scala b/measure/src/main/scala/org/apache/griffin/measure/process/engine/DqEngines.scala index cf5345420..7b0b34fa2 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/process/engine/DqEngines.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/process/engine/DqEngines.scala @@ -29,12 +29,12 @@ case class DqEngines(engines: Seq[DqEngine]) extends DqEngine { val persistOrder: List[PersistType] = List(MetricPersistType, RecordPersistType) - def initDataSources(dataSourceParams: Seq[DataSourceParam]): Unit = { - val dataSources = dataSourceParams.flatMap { param => - genDataSource(param) - } + def loadData(dataSources: Seq[DataSource]): Unit = { +// val dataSources = dataSourceParams.flatMap { param => +// genDataSource(param) +// } dataSources.foreach { ds => - ds.init + ds.loadData } } @@ -71,13 +71,13 @@ case class DqEngines(engines: Seq[DqEngine]) extends DqEngine { } } - def genDataSource(dataSourceParam: DataSourceParam): Option[DataSource] = { - val ret = engines.foldLeft(None: Option[DataSource]) { (dsOpt, engine) => - if (dsOpt.isEmpty) engine.genDataSource(dataSourceParam) else dsOpt - } - if (ret.isEmpty) warn(s"init data source warn: no dq engine support ${dataSourceParam}") - ret - } +// def genDataSource(dataSourceParam: DataSourceParam): Option[DirectDataSource] = { +// val ret = engines.foldLeft(None: Option[DirectDataSource]) { (dsOpt, engine) => +// if (dsOpt.isEmpty) engine.genDataSource(dataSourceParam) else dsOpt +// } +// if (ret.isEmpty) warn(s"init data source warn: no dq engine support ${dataSourceParam}") +// ret +// } def runRuleStep(ruleStep: ConcreteRuleStep): Boolean = { val ret = engines.foldLeft(false) { (done, engine) => diff --git a/measure/src/main/scala/org/apache/griffin/measure/process/engine/SparkSqlEngine.scala b/measure/src/main/scala/org/apache/griffin/measure/process/engine/SparkSqlEngine.scala index ac75b794d..0b95a35dd 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/process/engine/SparkSqlEngine.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/process/engine/SparkSqlEngine.scala @@ -32,9 +32,9 @@ import org.apache.spark.streaming.StreamingContext case class SparkSqlEngine(sqlContext: SQLContext, @transient ssc: StreamingContext ) extends DqEngine { - def genDataSource(dataSourceParam: DataSourceParam): Option[DataSource] = { - DataSourceFactory.genDataSource(sqlContext, ssc, dataSourceParam) - } +// def genDataSource(dataSourceParam: DataSourceParam): Option[DirectDataSource] = { +// DataSourceFactory.genDataSource(sqlContext, ssc, dataSourceParam) +// } def runRuleStep(ruleStep: ConcreteRuleStep): Boolean = { ruleStep match { diff --git a/measure/src/test/resources/config-test-accuracy-streaming.json b/measure/src/test/resources/config-test-accuracy-streaming.json new file mode 100644 index 000000000..c66a82dde --- /dev/null +++ b/measure/src/test/resources/config-test-accuracy-streaming.json @@ -0,0 +1,102 @@ +{ + "name": "accu_batch_test", + + "process.type": "streaming", + + "data.sources": [ + { + "name": "src", + "connectors": [ + { + "type": "kafka", + "version": "0.8", + "config": { + "kafka.config": { + "bootstrap.servers": "10.149.247.156:9092", + "group.id": "group1", + "auto.offset.reset": "smallest", + "auto.commit.enable": "false" + }, + "topics": "sss", + "key.type": "java.lang.String", + "value.type": "java.lang.String" + }, + "pre.proc": [ + { + "dsl.type": "df-opr", + "name": "source", + "rule": "from_json", + "persist.type": "cache" + } + ] + } + ], + "cache": { + "file.path": "hdfs://localhost/griffin/streaming/dump/source", + "info.path": "source", + "ready.time.interval": "10s", + "ready.time.delay": "0", + "time.range": ["-2m", "0"] + } + }, { + "name": "tgt", + "connectors": [ + { + "type": "kafka", + "version": "0.8", + "config": { + "kafka.config": { + "bootstrap.servers": "10.149.247.156:9092", + "group.id": "group1", + "auto.offset.reset": "smallest", + "auto.commit.enable": "false" + }, + "topics": "ttt", + "key.type": "java.lang.String", + "value.type": "java.lang.String" + }, + "pre.proc": [ + { + "dsl.type": "df-opr", + "name": "target", + "rule": "from_json", + "persist.type": "cache" + } + ] + } + ], + "cache": { + "file.path": "hdfs://localhost/griffin/streaming/dump/target", + "info.path": "target", + "ready.time.interval": "10s", + "ready.time.delay": "0", + "time.range": ["-2m", "0"] + } + } + ], + + "evaluateRule": { + "rules": [ + { + "dsl.type": "griffin-dsl", + "dq.type": "accuracy", + "rule": "source.name = target.name and source.age = target.age", + "details": { + "source": "source", + "target": "target", + "miss.records": { + "name": "miss.records", + "persist.type": "record" + }, + "accuracy": { + "name": "accu", + "persist.type": "metric" + }, + "miss": "miss_count", + "total": "total_count", + "matched": "matched_count" + } + } + ] + } +} \ No newline at end of file From 576a43e4046309008a480bfd3388c5ba53ac7f8e Mon Sep 17 00:00:00 2001 From: Lionel Liu Date: Tue, 12 Sep 2017 15:47:11 +0800 Subject: [PATCH 064/111] streaming calculation --- .../params/user/DataConnectorParam.scala | 2 +- .../config/params/user/RuleParam.scala | 68 ++++---- .../data/connector/DataConnector.scala | 72 ++++++++- .../data/connector/DataConnectorFactory.scala | 54 +++---- .../batch/AvroBatchDataConnector.scala | 8 +- .../batch/HiveBatchDataConnector.scala | 6 +- .../KafkaStreamingDataConnector.scala | 25 ++- .../KafkaStreamingStringDataConnector.scala | 65 ++++++++ .../streaming/StreamingDataConnector.scala | 2 +- .../measure/data/source/DataSource.scala | 2 +- .../measure/data/source/DataSourceCache.scala | 23 ++- .../data/source/DataSourceFactory.scala | 8 +- .../measure/persist/LoggerPersist.scala | 2 +- .../measure/process/BatchDqProcess.scala | 3 +- .../measure/process/StreamingDqProcess.scala | 44 +++--- .../measure/process/StreamingDqThread.scala | 148 ++++++++++++++++++ .../rules/adaptor/GriffinDslAdaptor.scala | 1 - .../rules/adaptor/RuleAdaptorGroup.scala | 13 +- .../rules/adaptor/SparkSqlAdaptor.scala | 1 - .../rules/preproc/PreProcRuleGenerator.scala | 68 ++++++++ .../griffin/measure/utils/HdfsUtil.scala | 2 +- .../config-test-accuracy-streaming.json | 30 +++- measure/src/test/resources/env-streaming.json | 1 + .../measure/process/JsonParseTest.scala | 46 ++++++ .../process/StreamingProcessTest.scala | 147 +++++++++++++++++ 25 files changed, 699 insertions(+), 142 deletions(-) create mode 100644 measure/src/main/scala/org/apache/griffin/measure/data/connector/streaming/KafkaStreamingStringDataConnector.scala create mode 100644 measure/src/main/scala/org/apache/griffin/measure/process/StreamingDqThread.scala create mode 100644 measure/src/main/scala/org/apache/griffin/measure/rules/preproc/PreProcRuleGenerator.scala create mode 100644 measure/src/test/scala/org/apache/griffin/measure/process/StreamingProcessTest.scala diff --git a/measure/src/main/scala/org/apache/griffin/measure/config/params/user/DataConnectorParam.scala b/measure/src/main/scala/org/apache/griffin/measure/config/params/user/DataConnectorParam.scala index 37877b9fd..a81999786 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/config/params/user/DataConnectorParam.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/config/params/user/DataConnectorParam.scala @@ -26,7 +26,7 @@ import org.apache.griffin.measure.config.params.Param case class DataConnectorParam( @JsonProperty("type") conType: String, @JsonProperty("version") version: String, @JsonProperty("config") config: Map[String, Any], - @JsonProperty("pre.proc") preProc: List[RuleParam] + @JsonProperty("pre.proc") preProc: List[Map[String, Any]] ) extends Param { diff --git a/measure/src/main/scala/org/apache/griffin/measure/config/params/user/RuleParam.scala b/measure/src/main/scala/org/apache/griffin/measure/config/params/user/RuleParam.scala index ccce71326..696f4fe20 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/config/params/user/RuleParam.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/config/params/user/RuleParam.scala @@ -1,34 +1,34 @@ -/* -Licensed to the Apache Software Foundation (ASF) under one -or more contributor license agreements. See the NOTICE file -distributed with this work for additional information -regarding copyright ownership. The ASF licenses this file -to you under the Apache License, Version 2.0 (the -"License"); you may not use this file except in compliance -with the License. You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, -software distributed under the License is distributed on an -"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -KIND, either express or implied. See the License for the -specific language governing permissions and limitations -under the License. -*/ -package org.apache.griffin.measure.config.params.user - -import com.fasterxml.jackson.annotation.{JsonInclude, JsonProperty} -import com.fasterxml.jackson.annotation.JsonInclude.Include -import org.apache.griffin.measure.config.params.Param - -@JsonInclude(Include.NON_NULL) -case class RuleParam(@JsonProperty("name") name: String, - @JsonProperty("dsl.type") dslType: String, - @JsonProperty("rule") rule: String, - @JsonProperty("persist.type") persistType: String, - @JsonProperty("dq.type") dqType: String, - @JsonProperty("details") details: Map[String, Any] - ) extends Param { - -} +///* +//Licensed to the Apache Software Foundation (ASF) under one +//or more contributor license agreements. See the NOTICE file +//distributed with this work for additional information +//regarding copyright ownership. The ASF licenses this file +//to you under the Apache License, Version 2.0 (the +//"License"); you may not use this file except in compliance +//with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +//Unless required by applicable law or agreed to in writing, +//software distributed under the License is distributed on an +//"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +//KIND, either express or implied. See the License for the +//specific language governing permissions and limitations +//under the License. +//*/ +//package org.apache.griffin.measure.config.params.user +// +//import com.fasterxml.jackson.annotation.{JsonInclude, JsonProperty} +//import com.fasterxml.jackson.annotation.JsonInclude.Include +//import org.apache.griffin.measure.config.params.Param +// +//@JsonInclude(Include.NON_NULL) +//case class RuleParam(@JsonProperty("name") name: String, +// @JsonProperty("dsl.type") dslType: String, +// @JsonProperty("rule") rule: String, +// @JsonProperty("persist.type") persistType: String, +// @JsonProperty("dq.type") dqType: String, +// @JsonProperty("details") details: Map[String, Any] +// ) extends Param { +// +//} diff --git a/measure/src/main/scala/org/apache/griffin/measure/data/connector/DataConnector.scala b/measure/src/main/scala/org/apache/griffin/measure/data/connector/DataConnector.scala index d8a51a7c8..f6b18d202 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/data/connector/DataConnector.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/data/connector/DataConnector.scala @@ -18,9 +18,17 @@ under the License. */ package org.apache.griffin.measure.data.connector +import java.util.concurrent.atomic.AtomicLong + +import org.apache.griffin.measure.config.params.user.DataConnectorParam import org.apache.griffin.measure.log.Loggable +import org.apache.griffin.measure.process.engine._ +import org.apache.griffin.measure.rules.adaptor.RuleAdaptorGroup +import org.apache.griffin.measure.rules.dsl._ +import org.apache.griffin.measure.rules.preproc.PreProcRuleGenerator import org.apache.spark.rdd.RDD -import org.apache.spark.sql.DataFrame +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.{DataFrame, SQLContext} trait DataConnector extends Loggable with Serializable { @@ -31,6 +39,66 @@ trait DataConnector extends Loggable with Serializable { def data(): Option[DataFrame] -// def preProcess(dfOpt: Option[DataFrame]): Option[DataFrame] + val dqEngines: DqEngines + + val dcParam: DataConnectorParam + + val sqlContext: SQLContext + + val id: String = DataConnectorIdGenerator.genId + + protected def suffix(ms: Long): String = s"${id}_${ms}" + protected def thisName(ms: Long): String = s"this_${suffix(ms)}" + + final val tmstColName = "__tmst" + + def preProcess(dfOpt: Option[DataFrame], ms: Long): Option[DataFrame] = { + val thisTable = thisName(ms) + val preProcRules = PreProcRuleGenerator.genPreProcRules(dcParam.preProc, suffix(ms)) + val names = PreProcRuleGenerator.getRuleNames(preProcRules).toSet + thisTable + + try { + dfOpt.flatMap { df => + // in data + df.registerTempTable(thisTable) + + // generate rule steps + val ruleSteps = RuleAdaptorGroup.genConcreteRuleSteps(preProcRules, DslType("spark-sql")) + + // run rules + dqEngines.runRuleSteps(ruleSteps) + + // out data + val outDf = sqlContext.table(thisTable) + + // drop temp table + names.foreach(name => sqlContext.dropTempTable(name)) + + // add tmst + val withTmstDf = outDf.withColumn(tmstColName, lit(ms)) + + Some(withTmstDf) + } + } catch { + case e: Throwable => { + error(s"preporcess of data connector [${id}] error: ${e.getMessage}") + None + } + } + + } + +} + +object DataConnectorIdGenerator { + private val counter: AtomicLong = new AtomicLong(0L) + private val head: String = "dc" + + def genId: String = { + s"${head}${increment}" + } + private def increment: Long = { + counter.incrementAndGet() + } } \ No newline at end of file diff --git a/measure/src/main/scala/org/apache/griffin/measure/data/connector/DataConnectorFactory.scala b/measure/src/main/scala/org/apache/griffin/measure/data/connector/DataConnectorFactory.scala index 2983f9c93..f8cde73ba 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/data/connector/DataConnectorFactory.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/data/connector/DataConnectorFactory.scala @@ -20,7 +20,8 @@ package org.apache.griffin.measure.data.connector import kafka.serializer.StringDecoder import org.apache.griffin.measure.config.params.user._ -import org.apache.griffin.measure.data.connector.streaming.{KafkaStreamingDataConnector, StreamingDataConnector} +import org.apache.griffin.measure.data.connector.streaming.{KafkaStreamingDataConnector, KafkaStreamingStringDataConnector, StreamingDataConnector} +import org.apache.griffin.measure.process.engine.{DqEngine, DqEngines} import org.apache.spark.rdd.RDD import org.apache.spark.sql.DataFrame @@ -46,22 +47,23 @@ object DataConnectorFactory { val TextRegex = """^(?i)text$""".r - def getDirectDataConnector(sqlContext: SQLContext, - ssc: StreamingContext, - dataConnectorParam: DataConnectorParam - ): Try[DataConnector] = { + def getDataConnector(sqlContext: SQLContext, + @transient ssc: StreamingContext, + dqEngines: DqEngines, + dataConnectorParam: DataConnectorParam + ): Try[DataConnector] = { val conType = dataConnectorParam.conType val version = dataConnectorParam.version val config = dataConnectorParam.config Try { conType match { - case HiveRegex() => HiveBatchDataConnector(sqlContext, config) - case AvroRegex() => AvroBatchDataConnector(sqlContext, config) + case HiveRegex() => HiveBatchDataConnector(sqlContext, dqEngines, dataConnectorParam) + case AvroRegex() => AvroBatchDataConnector(sqlContext, dqEngines, dataConnectorParam) case KafkaRegex() => { // val ksdcTry = getStreamingDataConnector(ssc, dataConnectorParam) // val cdcTry = getCacheDataConnector(sqlContext, dataConnectorParam.cache) // KafkaCacheDirectDataConnector(ksdcTry, cdcTry, dataConnectorParam) - getStreamingDataConnector(sqlContext, ssc, dataConnectorParam) + getStreamingDataConnector(sqlContext, ssc, dqEngines, dataConnectorParam) } case _ => throw new Exception("connector creation error!") } @@ -69,14 +71,15 @@ object DataConnectorFactory { } private def getStreamingDataConnector(sqlContext: SQLContext, - ssc: StreamingContext, + @transient ssc: StreamingContext, + dqEngines: DqEngines, dataConnectorParam: DataConnectorParam ): StreamingDataConnector = { + if (ssc == null) throw new Exception("streaming context is null!") val conType = dataConnectorParam.conType val version = dataConnectorParam.version - val config = dataConnectorParam.config conType match { - case KafkaRegex() => genKafkaDataConnector(sqlContext, ssc, config) + case KafkaRegex() => genKafkaDataConnector(sqlContext, ssc, dqEngines, dataConnectorParam) case _ => throw new Exception("streaming connector creation error!") } } @@ -98,37 +101,18 @@ object DataConnectorFactory { // } // private def genKafkaDataConnector(sqlContext: SQLContext, - ssc: StreamingContext, - config: Map[String, Any] + @transient ssc: StreamingContext, + dqEngines: DqEngines, + dataConnectorParam: DataConnectorParam ) = { + val config = dataConnectorParam.config val KeyType = "key.type" val ValueType = "value.type" val keyType = config.getOrElse(KeyType, "java.lang.String").toString val valueType = config.getOrElse(ValueType, "java.lang.String").toString (getClassTag(keyType), getClassTag(valueType)) match { case (ClassTag(k: Class[String]), ClassTag(v: Class[String])) => { - if (ssc == null) throw new Exception("streaming context is null! ") - new KafkaStreamingDataConnector(sqlContext, ssc, config) { - type K = String - type KD = StringDecoder - type V = String - type VD = StringDecoder - case class Value(value: K, _tmst: Long) {} - def createDStream(topicSet: Set[String]): InputDStream[(K, V)] = { - KafkaUtils.createDirectStream[K, V, KD, VD](ssc, kafkaConfig, topicSet) - } - def transform(rdd: RDD[(K, V)], ms: Long): Option[DataFrame] = { - val rdd1 = rdd.map(d => Value(d._2, ms)) - try { - Some(sqlContext.createDataFrame(rdd1, classOf[Value])) - } catch { - case e: Throwable => { - error(s"streaming data transform fails") - None - } - } - } - } + KafkaStreamingStringDataConnector(sqlContext, ssc, dqEngines, dataConnectorParam) } case _ => { throw new Exception("not supported type kafka data connector") diff --git a/measure/src/main/scala/org/apache/griffin/measure/data/connector/batch/AvroBatchDataConnector.scala b/measure/src/main/scala/org/apache/griffin/measure/data/connector/batch/AvroBatchDataConnector.scala index bf4923a0c..e603b903a 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/data/connector/batch/AvroBatchDataConnector.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/data/connector/batch/AvroBatchDataConnector.scala @@ -18,7 +18,9 @@ under the License. */ package org.apache.griffin.measure.data.connector.batch +import org.apache.griffin.measure.config.params.user.DataConnectorParam import org.apache.griffin.measure.data.connector._ +import org.apache.griffin.measure.process.engine.DqEngines import org.apache.griffin.measure.result._ import org.apache.griffin.measure.rule.{ExprValueUtil, RuleExprs} import org.apache.griffin.measure.utils.HdfsUtil @@ -28,8 +30,10 @@ import org.apache.spark.sql.{DataFrame, SQLContext} import scala.util.Try // data connector for avro file -case class AvroBatchDataConnector(sqlContext: SQLContext, config: Map[String, Any] - ) extends BatchDataConnector { +case class AvroBatchDataConnector(sqlContext: SQLContext, dqEngines: DqEngines, dcParam: DataConnectorParam + ) extends BatchDataConnector { + + val config = dcParam.config val FilePath = "file.path" val FileName = "file.name" diff --git a/measure/src/main/scala/org/apache/griffin/measure/data/connector/batch/HiveBatchDataConnector.scala b/measure/src/main/scala/org/apache/griffin/measure/data/connector/batch/HiveBatchDataConnector.scala index 2abca8220..81d7fe9b4 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/data/connector/batch/HiveBatchDataConnector.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/data/connector/batch/HiveBatchDataConnector.scala @@ -18,7 +18,9 @@ under the License. */ package org.apache.griffin.measure.data.connector.batch +import org.apache.griffin.measure.config.params.user.DataConnectorParam import org.apache.griffin.measure.data.connector._ +import org.apache.griffin.measure.process.engine.DqEngines import org.apache.griffin.measure.result._ import org.apache.griffin.measure.rule.{ExprValueUtil, RuleExprs} import org.apache.spark.rdd.RDD @@ -28,9 +30,11 @@ import org.apache.spark.sql.{DataFrame, SQLContext} import scala.util.{Success, Try} // data connector for hive -case class HiveBatchDataConnector(sqlContext: SQLContext, config: Map[String, Any] +case class HiveBatchDataConnector(sqlContext: SQLContext, dqEngines: DqEngines, dcParam: DataConnectorParam ) extends BatchDataConnector { + val config = dcParam.config + if (!sqlContext.isInstanceOf[HiveContext]) { throw new Exception("hive context not prepared!") } diff --git a/measure/src/main/scala/org/apache/griffin/measure/data/connector/streaming/KafkaStreamingDataConnector.scala b/measure/src/main/scala/org/apache/griffin/measure/data/connector/streaming/KafkaStreamingDataConnector.scala index cabdc34ba..964425dab 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/data/connector/streaming/KafkaStreamingDataConnector.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/data/connector/streaming/KafkaStreamingDataConnector.scala @@ -19,23 +19,17 @@ under the License. package org.apache.griffin.measure.data.connector.streaming import kafka.serializer.Decoder -import org.apache.griffin.measure.result.{DataInfo, TimeStampInfo} -import org.apache.griffin.measure.rule.{ExprValueUtil, RuleExprs} -import org.apache.griffin.measure.utils.JsonUtil -import org.apache.spark.rdd.RDD -import org.apache.spark.sql.SQLContext -import org.apache.spark.streaming.StreamingContext import org.apache.spark.streaming.dstream.InputDStream import scala.util.{Failure, Success, Try} -abstract class KafkaStreamingDataConnector(sqlContext: SQLContext, - @transient ssc: StreamingContext, - config: Map[String, Any] - ) extends StreamingDataConnector { +trait KafkaStreamingDataConnector extends StreamingDataConnector { + type KD <: Decoder[K] type VD <: Decoder[V] + val config = dcParam.config + val KafkaConfig = "kafka.config" val Topics = "topics" @@ -57,10 +51,12 @@ abstract class KafkaStreamingDataConnector(sqlContext: SQLContext, ds.foreachRDD((rdd, time) => { val ms = time.milliseconds - val dfOpt = transform(rdd, ms) + val dfOpt = transform(rdd) + + val preDfOpt = preProcess(dfOpt, ms) // save data frame - dataSourceCacheOpt.foreach(_.saveData(dfOpt, ms)) + dataSourceCacheOpt.foreach(_.saveData(preDfOpt, ms)) }) } @@ -70,4 +66,7 @@ abstract class KafkaStreamingDataConnector(sqlContext: SQLContext, } protected def createDStream(topicSet: Set[String]): InputDStream[(K, V)] -} \ No newline at end of file +} + + + diff --git a/measure/src/main/scala/org/apache/griffin/measure/data/connector/streaming/KafkaStreamingStringDataConnector.scala b/measure/src/main/scala/org/apache/griffin/measure/data/connector/streaming/KafkaStreamingStringDataConnector.scala new file mode 100644 index 000000000..e78734ede --- /dev/null +++ b/measure/src/main/scala/org/apache/griffin/measure/data/connector/streaming/KafkaStreamingStringDataConnector.scala @@ -0,0 +1,65 @@ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ +package org.apache.griffin.measure.data.connector.streaming + +import kafka.serializer.StringDecoder +import org.apache.griffin.measure.config.params.user.DataConnectorParam +import org.apache.griffin.measure.process.engine.DqEngines +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.types.{LongType, StringType, StructField, StructType} +import org.apache.spark.sql.{DataFrame, Row, SQLContext} +import org.apache.spark.streaming.StreamingContext +import org.apache.spark.streaming.dstream.InputDStream +import org.apache.spark.streaming.kafka.KafkaUtils +import org.apache.spark.sql.functions.lit + +case class KafkaStreamingStringDataConnector(sqlContext: SQLContext, + @transient ssc: StreamingContext, + dqEngines: DqEngines, + dcParam: DataConnectorParam + ) extends KafkaStreamingDataConnector { + type K = String + type KD = StringDecoder + type V = String + type VD = StringDecoder + + val valueColName = "value" + val schema = StructType(Array( + StructField("value", StringType) + )) + + def createDStream(topicSet: Set[String]): InputDStream[(K, V)] = { + KafkaUtils.createDirectStream[K, V, KD, VD](ssc, kafkaConfig, topicSet) + } + + def transform(rdd: RDD[(K, V)]): Option[DataFrame] = { + if (rdd.isEmpty) None else { + try { + val rowRdd = rdd.map(d => Row(d._2)) + val df = sqlContext.createDataFrame(rowRdd, schema) + Some(df) + } catch { + case e: Throwable => { + error(s"streaming data transform fails") + None + } + } + } + } +} diff --git a/measure/src/main/scala/org/apache/griffin/measure/data/connector/streaming/StreamingDataConnector.scala b/measure/src/main/scala/org/apache/griffin/measure/data/connector/streaming/StreamingDataConnector.scala index ef5799695..d9bb1f1d0 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/data/connector/streaming/StreamingDataConnector.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/data/connector/streaming/StreamingDataConnector.scala @@ -34,7 +34,7 @@ trait StreamingDataConnector extends DataConnector { protected def stream(): Try[InputDStream[(K, V)]] - def transform(rdd: RDD[(K, V)], ms: Long): Option[DataFrame] + def transform(rdd: RDD[(K, V)]): Option[DataFrame] def data(): Option[DataFrame] = None diff --git a/measure/src/main/scala/org/apache/griffin/measure/data/source/DataSource.scala b/measure/src/main/scala/org/apache/griffin/measure/data/source/DataSource.scala index ea071a9bf..739676f06 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/data/source/DataSource.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/data/source/DataSource.scala @@ -44,7 +44,7 @@ case class DataSource(name: String, df.registerTempTable(name) } case None => { - throw new Exception(s"load data source ${name} fails") + throw new Exception(s"load data source [${name}] fails") } } } diff --git a/measure/src/main/scala/org/apache/griffin/measure/data/source/DataSourceCache.scala b/measure/src/main/scala/org/apache/griffin/measure/data/source/DataSourceCache.scala index ad1f37720..c382dee7b 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/data/source/DataSourceCache.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/data/source/DataSourceCache.scala @@ -92,9 +92,6 @@ case class DataSourceCache(sqlContext: SQLContext, param: Map[String, Any], HdfsFileDumpUtil.dump(dataFilePath, dataRdd, rowSepLiteral) } else false - // submit ms - submitCacheTime(ms) - submitReadyTime(ms) } catch { case e: Throwable => error(s"save data error: ${e.getMessage}") } finally { @@ -107,6 +104,9 @@ case class DataSourceCache(sqlContext: SQLContext, param: Map[String, Any], } } + // submit cache time and ready time + submitCacheTime(ms) + submitReadyTime(ms) } def readData(): Option[DataFrame] = { @@ -127,15 +127,14 @@ case class DataSourceCache(sqlContext: SQLContext, param: Map[String, Any], if (partitionPaths.isEmpty) { None } else { -// val filePaths = partitionPaths.mkString(",") -// val rdd: RDD[String] = sqlContext.sparkContext.textFile(filePaths) -// -// // decode data -// rdd.flatMap { row => -// decode(row) -// } - val df = sqlContext.read.text(partitionPaths: _*) - Some(df) + try { + Some(sqlContext.read.json(partitionPaths: _*)) + } catch { + case e: Throwable => { + error(s"read data source cache error: ${e.getMessage}") + None + } + } } } diff --git a/measure/src/main/scala/org/apache/griffin/measure/data/source/DataSourceFactory.scala b/measure/src/main/scala/org/apache/griffin/measure/data/source/DataSourceFactory.scala index 328279faf..a713f3caf 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/data/source/DataSourceFactory.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/data/source/DataSourceFactory.scala @@ -23,6 +23,7 @@ import org.apache.griffin.measure.data.connector.batch.BatchDataConnector import org.apache.griffin.measure.data.connector.streaming.StreamingDataConnector import org.apache.griffin.measure.data.connector.{DataConnector, DataConnectorFactory} import org.apache.griffin.measure.log.Loggable +import org.apache.griffin.measure.process.engine.{DqEngine, DqEngines} import org.apache.spark.sql.SQLContext import org.apache.spark.streaming.StreamingContext @@ -34,15 +35,16 @@ object DataSourceFactory extends Loggable { val TextRegex = """^(?i)text$""".r val AvroRegex = """^(?i)avro$""".r - def genDataSources(sqlContext: SQLContext, ssc: StreamingContext, + def genDataSources(sqlContext: SQLContext, ssc: StreamingContext, dqEngines: DqEngines, dataSourceParams: Seq[DataSourceParam], metricName: String): Seq[DataSource] = { dataSourceParams.zipWithIndex.flatMap { pair => val (param, index) = pair - genDataSource(sqlContext, ssc, param, metricName, index) + genDataSource(sqlContext, ssc, dqEngines, param, metricName, index) } } private def genDataSource(sqlContext: SQLContext, ssc: StreamingContext, + dqEngines: DqEngines, dataSourceParam: DataSourceParam, metricName: String, index: Int ): Option[DataSource] = { @@ -50,7 +52,7 @@ object DataSourceFactory extends Loggable { val connectorParams = dataSourceParam.connectors val cacheParam = dataSourceParam.cache val dataConnectors = connectorParams.flatMap { connectorParam => - DataConnectorFactory.getDirectDataConnector(sqlContext, ssc, connectorParam) match { + DataConnectorFactory.getDataConnector(sqlContext, ssc, dqEngines, connectorParam) match { case Success(connector) => Some(connector) case _ => None } diff --git a/measure/src/main/scala/org/apache/griffin/measure/persist/LoggerPersist.scala b/measure/src/main/scala/org/apache/griffin/measure/persist/LoggerPersist.scala index 8781c8315..07774b720 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/persist/LoggerPersist.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/persist/LoggerPersist.scala @@ -34,7 +34,7 @@ case class LoggerPersist(config: Map[String, Any], metricName: String, timeStamp def available(): Boolean = true def start(msg: String): Unit = { - println(s"[${timeStamp}] ${metricName} start") + println(s"[${timeStamp}] ${metricName} start: ${msg}") } def finish(): Unit = { println(s"[${timeStamp}] ${metricName} finish") diff --git a/measure/src/main/scala/org/apache/griffin/measure/process/BatchDqProcess.scala b/measure/src/main/scala/org/apache/griffin/measure/process/BatchDqProcess.scala index e500a892a..b8a388131 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/process/BatchDqProcess.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/process/BatchDqProcess.scala @@ -78,7 +78,8 @@ case class BatchDqProcess(allParam: AllParam) extends DqProcess { val dqEngines = DqEngineFactory.genDqEngines(sqlContext, null) // generate data sources - val dataSources = DataSourceFactory.genDataSources(sqlContext, null, userParam.dataSources, metricName) + val dataSources = DataSourceFactory.genDataSources(sqlContext, null, dqEngines, userParam.dataSources, metricName) + dataSources.foreach(_.init) // init data sources dqEngines.loadData(dataSources) diff --git a/measure/src/main/scala/org/apache/griffin/measure/process/StreamingDqProcess.scala b/measure/src/main/scala/org/apache/griffin/measure/process/StreamingDqProcess.scala index 107a574bd..428a44b54 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/process/StreamingDqProcess.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/process/StreamingDqProcess.scala @@ -96,30 +96,34 @@ case class StreamingDqProcess(allParam: AllParam) extends DqProcess { val dqEngines = DqEngineFactory.genDqEngines(sqlContext, ssc) // generate data sources - val dataSources = DataSourceFactory.genDataSources(sqlContext, null, userParam.dataSources, metricName) + val dataSources = DataSourceFactory.genDataSources(sqlContext, ssc, dqEngines, userParam.dataSources, metricName) + dataSources.foreach(_.init) - // init data sources - dqEngines.loadData(dataSources) - - // generate rule steps - val ruleSteps = RuleAdaptorGroup.genConcreteRuleSteps(userParam.evaluateRuleParam) + // process thread + val dqThread = StreamingDqThread(dqEngines, dataSources, userParam.evaluateRuleParam, persist) - // run rules - dqEngines.runRuleSteps(ruleSteps) - - // persist results - dqEngines.persistAllResults(ruleSteps, persist) + // init data sources +// dqEngines.loadData(dataSources) +// +// // generate rule steps +// val ruleSteps = RuleAdaptorGroup.genConcreteRuleSteps(userParam.evaluateRuleParam) +// +// // run rules +// dqEngines.runRuleSteps(ruleSteps) +// +// // persist results +// dqEngines.persistAllResults(ruleSteps, persist) // end time - val endTime = new Date().getTime - persist.log(endTime, s"process using time: ${endTime - startTime} ms") - -// val processInterval = TimeUtil.milliseconds(sparkParam.processInterval) match { -// case Some(interval) => interval -// case _ => throw new Exception("invalid batch interval") -// } -// val process = TimingProcess(processInterval, streamingAccuracyProcess) -// process.startup() +// val endTime = new Date().getTime +// persist.log(endTime, s"process using time: ${endTime - startTime} ms") + + val processInterval = TimeUtil.milliseconds(sparkParam.processInterval) match { + case Some(interval) => interval + case _ => throw new Exception("invalid batch interval") + } + val process = TimingProcess(processInterval, dqThread) + process.startup() ssc.start() ssc.awaitTermination() diff --git a/measure/src/main/scala/org/apache/griffin/measure/process/StreamingDqThread.scala b/measure/src/main/scala/org/apache/griffin/measure/process/StreamingDqThread.scala new file mode 100644 index 000000000..5f3c90a6d --- /dev/null +++ b/measure/src/main/scala/org/apache/griffin/measure/process/StreamingDqThread.scala @@ -0,0 +1,148 @@ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ +package org.apache.griffin.measure.process + +import java.util.Date +import java.util.concurrent.TimeUnit + +import org.apache.griffin.measure.cache.info.{InfoCacheInstance, TimeInfoCache} +import org.apache.griffin.measure.config.params.user.EvaluateRuleParam +import org.apache.griffin.measure.data.source.DataSource +import org.apache.griffin.measure.log.Loggable +import org.apache.griffin.measure.persist.Persist +import org.apache.griffin.measure.process.engine.DqEngines +import org.apache.griffin.measure.rules.adaptor.RuleAdaptorGroup + +case class StreamingDqThread(dqEngines: DqEngines, + dataSources: Seq[DataSource], + evaluateRuleParam: EvaluateRuleParam, + persist: Persist + ) extends Runnable with Loggable { + + val lock = InfoCacheInstance.genLock("process") + + def run(): Unit = { + val updateTimeDate = new Date() + val updateTime = updateTimeDate.getTime + println(s"===== [${updateTimeDate}] process begins =====") + val locked = lock.lock(5, TimeUnit.SECONDS) + if (locked) { + try { + + val st = new Date().getTime + persist.log(st, s"starting process ...") + + TimeInfoCache.startTimeInfoCache + + // init data sources + dqEngines.loadData(dataSources) + + // generate rule steps + val ruleSteps = RuleAdaptorGroup.genConcreteRuleSteps(evaluateRuleParam) + + // run rules + dqEngines.runRuleSteps(ruleSteps) + + val ct = new Date().getTime + persist.log(ct, s"calculation using time: ${ct - st} ms") + + // persist results + dqEngines.persistAllResults(ruleSteps, persist) + + TimeInfoCache.endTimeInfoCache + + val et = new Date().getTime + persist.log(et, s"persist using time: ${et - ct} ms") + + } catch { + case e: Throwable => error(s"process error: ${e.getMessage}") + } finally { + lock.unlock() + } + } else { + println(s"===== [${updateTimeDate}] process ignores =====") + } + val endTime = new Date().getTime + println(s"===== [${updateTimeDate}] process ends, using ${endTime - updateTime} ms =====") + } + + // clean old data and old result cache +// def cleanData(): Unit = { +// try { +// sourceDataConnector.cleanOldData +// targetDataConnector.cleanOldData +// +// val cleanTime = TimeInfoCache.getCleanTime +// cacheResultProcesser.refresh(cleanTime) +// } catch { +// case e: Throwable => error(s"clean data error: ${e.getMessage}") +// } +// } +// +// // calculate accuracy between source data and target data +// private def accuracy(sourceData: RDD[(Product, (Map[String, Any], Map[String, Any]))], +// targetData: RDD[(Product, (Map[String, Any], Map[String, Any]))], +// ruleAnalyzer: RuleAnalyzer) = { +// // 1. cogroup +// val allKvs = sourceData.cogroup(targetData) +// +// // 2. accuracy calculation +// val (accuResult, missingRdd, matchedRdd) = AccuracyCore.accuracy(allKvs, ruleAnalyzer) +// +// (accuResult, missingRdd, matchedRdd) +// } +// +// private def reorgByTimeGroup(rdd: RDD[(Product, (Map[String, Any], Map[String, Any]))] +// ): RDD[(Long, (Product, (Map[String, Any], Map[String, Any])))] = { +// rdd.flatMap { row => +// val (key, (value, info)) = row +// val b: Option[(Long, (Product, (Map[String, Any], Map[String, Any])))] = info.get(TimeStampInfo.key) match { +// case Some(t: Long) => Some((t, row)) +// case _ => None +// } +// b +// } +// } +// +// // convert data into a string +// def record2String(rec: (Product, (Map[String, Any], Map[String, Any])), dataPersist: Iterable[Expr], infoPersist: Iterable[Expr]): String = { +// val (key, (data, info)) = rec +// val persistData = getPersistMap(data, dataPersist) +// val persistInfo = info.mapValues { value => +// value match { +// case vd: Map[String, Any] => getPersistMap(vd, infoPersist) +// case v => v +// } +// }.map(identity) +// s"${persistData} [${persistInfo}]" +// } +// +// // get the expr value map of the persist expressions +// private def getPersistMap(data: Map[String, Any], persist: Iterable[Expr]): Map[String, Any] = { +// val persistMap = persist.map(e => (e._id, e.desc)).toMap +// data.flatMap { pair => +// val (k, v) = pair +// persistMap.get(k) match { +// case Some(d) => Some((d -> v)) +// case _ => None +// } +// } +// } + +} diff --git a/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/GriffinDslAdaptor.scala b/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/GriffinDslAdaptor.scala index d84a76ca2..2ff7c816a 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/GriffinDslAdaptor.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/GriffinDslAdaptor.scala @@ -18,7 +18,6 @@ under the License. */ package org.apache.griffin.measure.rules.adaptor -import org.apache.griffin.measure.config.params.user.RuleParam import org.apache.griffin.measure.rules.dsl._ import org.apache.griffin.measure.rules.dsl.analyzer._ import org.apache.griffin.measure.rules.dsl.expr._ diff --git a/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/RuleAdaptorGroup.scala b/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/RuleAdaptorGroup.scala index 3b9f5a5bd..1bf970f36 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/RuleAdaptorGroup.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/RuleAdaptorGroup.scala @@ -18,7 +18,7 @@ under the License. */ package org.apache.griffin.measure.rules.adaptor -import org.apache.griffin.measure.config.params.user.{EvaluateRuleParam, RuleParam} +import org.apache.griffin.measure.config.params.user._ import org.apache.griffin.measure.rules.dsl._ import org.apache.griffin.measure.rules.step._ import org.apache.spark.sql.SQLContext @@ -46,11 +46,11 @@ object RuleAdaptorGroup { } } - private def genRuleAdaptor(dslType: DslType, dataSourceNames: Seq[String]): Option[RuleAdaptor] = { + private def genRuleAdaptor(dslType: DslType, dsNames: Seq[String]): Option[RuleAdaptor] = { dslType match { case SparkSqlType => Some(SparkSqlAdaptor()) case DfOprType => Some(DataFrameOprAdaptor()) - case GriffinDslType => Some(GriffinDslAdaptor(dataSourceNames, functionNames)) + case GriffinDslType => Some(GriffinDslAdaptor(dsNames, functionNames)) case _ => None } } @@ -75,9 +75,13 @@ object RuleAdaptorGroup { val dslTypeStr = if (evaluateRuleParam.dslType == null) "" else evaluateRuleParam.dslType val defaultDslType = DslType(dslTypeStr) val ruleParams = evaluateRuleParam.rules + genConcreteRuleSteps(ruleParams, defaultDslType) + } + + def genConcreteRuleSteps(ruleParams: Seq[Map[String, Any]], defDslType: DslType): Seq[ConcreteRuleStep] = { val (steps, dsNames) = ruleParams.foldLeft((Seq[ConcreteRuleStep](), dataSourceNames)) { (res, param) => val (preSteps, preNames) = res - val dslType = getDslType(param, defaultDslType) + val dslType = getDslType(param, defDslType) val (curSteps, curNames) = genRuleAdaptor(dslType, preNames) match { case Some(ruleAdaptor) => (ruleAdaptor.genConcreteRuleStep(param), preNames ++ ruleAdaptor.getTempSourceNames(param)) case _ => (Nil, preNames) @@ -88,5 +92,4 @@ object RuleAdaptorGroup { } - } diff --git a/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/SparkSqlAdaptor.scala b/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/SparkSqlAdaptor.scala index f4357a429..d6890093e 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/SparkSqlAdaptor.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/SparkSqlAdaptor.scala @@ -18,7 +18,6 @@ under the License. */ package org.apache.griffin.measure.rules.adaptor -import org.apache.griffin.measure.config.params.user.RuleParam import org.apache.griffin.measure.rules.dsl.PersistType import org.apache.griffin.measure.rules.step._ diff --git a/measure/src/main/scala/org/apache/griffin/measure/rules/preproc/PreProcRuleGenerator.scala b/measure/src/main/scala/org/apache/griffin/measure/rules/preproc/PreProcRuleGenerator.scala new file mode 100644 index 000000000..d63567cad --- /dev/null +++ b/measure/src/main/scala/org/apache/griffin/measure/rules/preproc/PreProcRuleGenerator.scala @@ -0,0 +1,68 @@ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ +package org.apache.griffin.measure.rules.preproc + +object PreProcRuleGenerator { + + val _name = "name" + + def genPreProcRules(rules: Seq[Map[String, Any]], suffix: String): Seq[Map[String, Any]] = { + rules.map { rule => + genPreProcRule(rule, suffix) + } + } + + def getRuleNames(rules: Seq[Map[String, Any]]): Seq[String] = { + rules.flatMap { rule => + rule.get(_name) match { + case Some(s: String) => Some(s) + case _ => None + } + } + } + + private def genPreProcRule(param: Map[String, Any], suffix: String + ): Map[String, Any] = { + val keys = param.keys + keys.foldLeft(param) { (map, key) => + map.get(key) match { + case Some(s: String) => map + (key -> genNewString(s, suffix)) + case Some(subMap: Map[String, Any]) => map + (key -> genPreProcRule(subMap, suffix)) + case Some(arr: Seq[_]) => map + (key -> genPreProcRule(arr, suffix)) + case _ => map + } + } + } + + private def genPreProcRule(paramArr: Seq[Any], suffix: String): Seq[Any] = { + paramArr.foldLeft(Nil: Seq[Any]) { (res, param) => + param match { + case s: String => res :+ genNewString(s, suffix) + case map: Map[String, Any] => res :+ genPreProcRule(map, suffix) + case arr: Seq[_] => res :+ genPreProcRule(arr, suffix) + case _ => res :+ param + } + } + } + + private def genNewString(str: String, suffix: String): String = { + str.replaceAll("""\$\{(.*)\}""", s"$$1_${suffix}") + } + +} diff --git a/measure/src/main/scala/org/apache/griffin/measure/utils/HdfsUtil.scala b/measure/src/main/scala/org/apache/griffin/measure/utils/HdfsUtil.scala index 6dd54b7f6..079359275 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/utils/HdfsUtil.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/utils/HdfsUtil.scala @@ -27,7 +27,7 @@ object HdfsUtil { private val conf = new Configuration() conf.set("dfs.support.append", "true") -// conf.set("fs.defaultFS", "hdfs://localhost") // debug @localhost + conf.set("fs.defaultFS", "hdfs://localhost") // debug @localhost private val dfs = FileSystem.get(conf) diff --git a/measure/src/test/resources/config-test-accuracy-streaming.json b/measure/src/test/resources/config-test-accuracy-streaming.json index c66a82dde..95c3ee12b 100644 --- a/measure/src/test/resources/config-test-accuracy-streaming.json +++ b/measure/src/test/resources/config-test-accuracy-streaming.json @@ -1,11 +1,11 @@ { - "name": "accu_batch_test", + "name": "accu_streaming", "process.type": "streaming", "data.sources": [ { - "name": "src", + "name": "source", "connectors": [ { "type": "kafka", @@ -24,9 +24,17 @@ "pre.proc": [ { "dsl.type": "df-opr", - "name": "source", + "name": "${s1}", "rule": "from_json", - "persist.type": "cache" + "persist.type": "cache", + "details": { + "df.name": "${this}" + } + }, + { + "dsl.type": "spark-sql", + "name": "${this}", + "rule": "select name, age from ${s1}" } ] } @@ -39,7 +47,7 @@ "time.range": ["-2m", "0"] } }, { - "name": "tgt", + "name": "target", "connectors": [ { "type": "kafka", @@ -58,9 +66,17 @@ "pre.proc": [ { "dsl.type": "df-opr", - "name": "target", + "name": "${t1}", "rule": "from_json", - "persist.type": "cache" + "persist.type": "cache", + "details": { + "df.name": "${this}" + } + }, + { + "dsl.type": "spark-sql", + "name": "${this}", + "rule": "select name, age from ${t1}" } ] } diff --git a/measure/src/test/resources/env-streaming.json b/measure/src/test/resources/env-streaming.json index 42b4aa97d..a01348f82 100644 --- a/measure/src/test/resources/env-streaming.json +++ b/measure/src/test/resources/env-streaming.json @@ -5,6 +5,7 @@ "batch.interval": "2s", "process.interval": "10s", "config": { + "spark.master": "local[*]", "spark.task.maxFailures": 5, "spark.streaming.kafkaMaxRatePerPartition": 1000, "spark.streaming.concurrentJobs": 4, diff --git a/measure/src/test/scala/org/apache/griffin/measure/process/JsonParseTest.scala b/measure/src/test/scala/org/apache/griffin/measure/process/JsonParseTest.scala index e68a4c9fa..11a9a1550 100644 --- a/measure/src/test/scala/org/apache/griffin/measure/process/JsonParseTest.scala +++ b/measure/src/test/scala/org/apache/griffin/measure/process/JsonParseTest.scala @@ -483,4 +483,50 @@ class JsonParseTest extends FunSuite with Matchers with BeforeAndAfter with Logg val functionNames = functions.map(_.getString(0)).collect functionNames.foreach(println) } + + test ("test text file read") { + val partitionPaths = Seq[String]( + "hdfs://localhost/griffin/streaming/dump/source/418010/25080625/1504837518000", + "hdfs://localhost/griffin/streaming/dump/target/418010/25080625/1504837518000") + val df = sqlContext.read.json(partitionPaths: _*) + df.printSchema() + df.show(10) + } + + test ("list paths") { + val filePath = "hdfs://localhost/griffin/streaming/dump/source" + val partitionRanges = List[(Long, Long)]((0, 0), (-2, 0)) + val partitionPaths = listPathsBetweenRanges(filePath :: Nil, partitionRanges) + println(partitionPaths) + } + + private def listPathsBetweenRanges(paths: List[String], + partitionRanges: List[(Long, Long)] + ): List[String] = { + partitionRanges match { + case Nil => paths + case head :: tail => { + val (lb, ub) = head + val curPaths = paths.flatMap { path => + val names = HdfsUtil.listSubPaths(path, "dir").toList + println(names) + names.filter { name => + str2Long(name) match { + case Some(t) => (t >= lb) && (t <= ub) + case _ => false + } + }.map(HdfsUtil.getHdfsFilePath(path, _)) + } + listPathsBetweenRanges(curPaths, tail) + } + } + } + + private def str2Long(str: String): Option[Long] = { + try { + Some(str.toLong) + } catch { + case e: Throwable => None + } + } } \ No newline at end of file diff --git a/measure/src/test/scala/org/apache/griffin/measure/process/StreamingProcessTest.scala b/measure/src/test/scala/org/apache/griffin/measure/process/StreamingProcessTest.scala new file mode 100644 index 000000000..cbbcedf70 --- /dev/null +++ b/measure/src/test/scala/org/apache/griffin/measure/process/StreamingProcessTest.scala @@ -0,0 +1,147 @@ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ +package org.apache.griffin.measure.process + +import org.apache.griffin.measure.algo.ProcessType +import org.apache.griffin.measure.config.params._ +import org.apache.griffin.measure.config.params.env._ +import org.apache.griffin.measure.config.params.user._ +import org.apache.griffin.measure.config.reader.ParamReaderFactory +import org.apache.griffin.measure.config.validator.AllParamValidator +import org.apache.griffin.measure.log.Loggable +import org.apache.griffin.measure.persist.PersistThreadPool +import org.junit.runner.RunWith +import org.scalatest.junit.JUnitRunner +import org.scalatest.{BeforeAndAfter, FunSuite, Matchers} + +import scala.util.{Failure, Success, Try} + +@RunWith(classOf[JUnitRunner]) +class StreamingProcessTest extends FunSuite with Matchers with BeforeAndAfter with Loggable { + + val envFile = "src/test/resources/env-streaming.json" + val confFile = "src/test/resources/config-test-accuracy-streaming.json" +// val confFile = "src/test/resources/config-test-accuracy.json" + + val envFsType = "local" + val userFsType = "local" + + val args = Array(envFile, confFile) + + var allParam: AllParam = _ + + before { + // read param files + val envParam = readParamFile[EnvParam](envFile, envFsType) match { + case Success(p) => p + case Failure(ex) => { + error(ex.getMessage) + sys.exit(-2) + } + } + val userParam = readParamFile[UserParam](confFile, userFsType) match { + case Success(p) => p + case Failure(ex) => { + error(ex.getMessage) + sys.exit(-2) + } + } + allParam = AllParam(envParam, userParam) + + // validate param files + validateParams(allParam) match { + case Failure(ex) => { + error(ex.getMessage) + sys.exit(-3) + } + case _ => { + info("params validation pass") + } + } + } + + test ("streaming process") { + val procType = allParam.userParam.procType + val proc: DqProcess = procType match { + case ProcessType.batch() => BatchDqProcess(allParam) + case ProcessType.streaming() => StreamingDqProcess(allParam) + case _ => { + error(s"${procType} is unsupported process type!") + sys.exit(-4) + } + } + + // process init + proc.init match { + case Success(_) => { + info("process init success") + } + case Failure(ex) => { + error(s"process init error: ${ex.getMessage}") + shutdown + sys.exit(-5) + } + } + + // process run + proc.run match { + case Success(_) => { + info("process run success") + } + case Failure(ex) => { + error(s"process run error: ${ex.getMessage}") + + if (proc.retriable) { + throw ex + } else { + shutdown + sys.exit(-5) + } + } + } + + // process end + proc.end match { + case Success(_) => { + info("process end success") + } + case Failure(ex) => { + error(s"process end error: ${ex.getMessage}") + shutdown + sys.exit(-5) + } + } + + shutdown + } + + private def readParamFile[T <: Param](file: String, fsType: String)(implicit m : Manifest[T]): Try[T] = { + val paramReader = ParamReaderFactory.getParamReader(file, fsType) + paramReader.readConfig[T] + } + + private def validateParams(allParam: AllParam): Try[Boolean] = { + val allParamValidator = AllParamValidator() + allParamValidator.validate(allParam) + } + + private def shutdown(): Unit = { + PersistThreadPool.shutdown + } +} From 3c61801b55605ae0949ccb452284d3e760e865a9 Mon Sep 17 00:00:00 2001 From: Lionel Liu Date: Tue, 12 Sep 2017 17:34:15 +0800 Subject: [PATCH 065/111] proc type --- .../apache/griffin/measure/Application.scala | 6 +-- .../griffin/measure/algo/ProcessType.scala | 27 ++++++++-- .../data/connector/DataConnector.scala | 9 +++- .../measure/process/BatchDqProcess.scala | 3 +- .../measure/process/StreamingDqThread.scala | 13 ++--- .../rules/adaptor/DataFrameOprAdaptor.scala | 3 +- .../rules/adaptor/GriffinDslAdaptor.scala | 53 ++++++++++++++----- .../measure/rules/adaptor/RuleAdaptor.scala | 4 ++ .../rules/adaptor/RuleAdaptorGroup.scala | 17 +++--- .../rules/adaptor/SparkSqlAdaptor.scala | 3 +- .../config-test-accuracy-streaming.json | 3 +- .../test/resources/config-test-profiling.json | 12 +++++ .../measure/process/BatchProcessTest.scala | 8 +-- .../measure/process/JsonParseTest.scala | 2 +- .../process/StreamingProcessTest.scala | 8 +-- 15 files changed, 123 insertions(+), 48 deletions(-) diff --git a/measure/src/main/scala/org/apache/griffin/measure/Application.scala b/measure/src/main/scala/org/apache/griffin/measure/Application.scala index 5e7e8dbc0..dd5742bad 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/Application.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/Application.scala @@ -83,10 +83,10 @@ object Application extends Loggable { // choose algorithm // val dqType = allParam.userParam.dqType - val procType = allParam.userParam.procType + val procType = ProcessType(allParam.userParam.procType) val proc: DqProcess = procType match { - case ProcessType.batch() => BatchDqProcess(allParam) - case ProcessType.streaming() => StreamingDqProcess(allParam) + case BatchProcessType => BatchDqProcess(allParam) + case StreamingProcessType => StreamingDqProcess(allParam) case _ => { error(s"${procType} is unsupported process type!") sys.exit(-4) diff --git a/measure/src/main/scala/org/apache/griffin/measure/algo/ProcessType.scala b/measure/src/main/scala/org/apache/griffin/measure/algo/ProcessType.scala index 5a85c7c00..2f2dc7ac6 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/algo/ProcessType.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/algo/ProcessType.scala @@ -18,9 +18,30 @@ under the License. */ package org.apache.griffin.measure.algo -object ProcessType { +import scala.util.matching.Regex + +sealed trait ProcessType { + val regex: Regex + val desc: String +} - val batch = """^(?i)batch$""".r - val streaming = """^(?i)streaming$""".r +object ProcessType { + private val procTypes: List[ProcessType] = List(BatchProcessType, StreamingProcessType) + def apply(ptn: String): ProcessType = { + procTypes.filter(tp => ptn match { + case tp.regex() => true + case _ => false + }).headOption.getOrElse(BatchProcessType) + } + def unapply(pt: ProcessType): Option[String] = Some(pt.desc) +} +final case object BatchProcessType extends ProcessType { + val regex = """^(?i)batch$""".r + val desc = "batch" } + +final case object StreamingProcessType extends ProcessType { + val regex = """^(?i)streaming$""".r + val desc = "streaming" +} \ No newline at end of file diff --git a/measure/src/main/scala/org/apache/griffin/measure/data/connector/DataConnector.scala b/measure/src/main/scala/org/apache/griffin/measure/data/connector/DataConnector.scala index f6b18d202..acb52fd46 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/data/connector/DataConnector.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/data/connector/DataConnector.scala @@ -20,6 +20,7 @@ package org.apache.griffin.measure.data.connector import java.util.concurrent.atomic.AtomicLong +import org.apache.griffin.measure.algo._ import org.apache.griffin.measure.config.params.user.DataConnectorParam import org.apache.griffin.measure.log.Loggable import org.apache.griffin.measure.process.engine._ @@ -50,7 +51,7 @@ trait DataConnector extends Loggable with Serializable { protected def suffix(ms: Long): String = s"${id}_${ms}" protected def thisName(ms: Long): String = s"this_${suffix(ms)}" - final val tmstColName = "__tmst" + final val tmstColName = GroupByColumn.tmst def preProcess(dfOpt: Option[DataFrame], ms: Long): Option[DataFrame] = { val thisTable = thisName(ms) @@ -63,7 +64,7 @@ trait DataConnector extends Loggable with Serializable { df.registerTempTable(thisTable) // generate rule steps - val ruleSteps = RuleAdaptorGroup.genConcreteRuleSteps(preProcRules, DslType("spark-sql")) + val ruleSteps = RuleAdaptorGroup.genConcreteRuleSteps(preProcRules, DslType("spark-sql"), BatchProcessType) // run rules dqEngines.runRuleSteps(ruleSteps) @@ -101,4 +102,8 @@ object DataConnectorIdGenerator { private def increment: Long = { counter.incrementAndGet() } +} + +object GroupByColumn { + val tmst = "__tmst" } \ No newline at end of file diff --git a/measure/src/main/scala/org/apache/griffin/measure/process/BatchDqProcess.scala b/measure/src/main/scala/org/apache/griffin/measure/process/BatchDqProcess.scala index b8a388131..356a50436 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/process/BatchDqProcess.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/process/BatchDqProcess.scala @@ -20,6 +20,7 @@ package org.apache.griffin.measure.process import java.util.Date +import org.apache.griffin.measure.algo._ import org.apache.griffin.measure.config.params._ import org.apache.griffin.measure.config.params.env._ import org.apache.griffin.measure.config.params.user._ @@ -85,7 +86,7 @@ case class BatchDqProcess(allParam: AllParam) extends DqProcess { dqEngines.loadData(dataSources) // generate rule steps - val ruleSteps = RuleAdaptorGroup.genConcreteRuleSteps(userParam.evaluateRuleParam) + val ruleSteps = RuleAdaptorGroup.genConcreteRuleSteps(userParam.evaluateRuleParam, BatchProcessType) // run rules dqEngines.runRuleSteps(ruleSteps) diff --git a/measure/src/main/scala/org/apache/griffin/measure/process/StreamingDqThread.scala b/measure/src/main/scala/org/apache/griffin/measure/process/StreamingDqThread.scala index 5f3c90a6d..5a89bfb5b 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/process/StreamingDqThread.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/process/StreamingDqThread.scala @@ -21,6 +21,7 @@ package org.apache.griffin.measure.process import java.util.Date import java.util.concurrent.TimeUnit +import org.apache.griffin.measure.algo._ import org.apache.griffin.measure.cache.info.{InfoCacheInstance, TimeInfoCache} import org.apache.griffin.measure.config.params.user.EvaluateRuleParam import org.apache.griffin.measure.data.source.DataSource @@ -32,7 +33,7 @@ import org.apache.griffin.measure.rules.adaptor.RuleAdaptorGroup case class StreamingDqThread(dqEngines: DqEngines, dataSources: Seq[DataSource], evaluateRuleParam: EvaluateRuleParam, - persist: Persist + appPersist: Persist ) extends Runnable with Loggable { val lock = InfoCacheInstance.genLock("process") @@ -46,7 +47,7 @@ case class StreamingDqThread(dqEngines: DqEngines, try { val st = new Date().getTime - persist.log(st, s"starting process ...") + appPersist.log(st, s"starting process ...") TimeInfoCache.startTimeInfoCache @@ -54,21 +55,21 @@ case class StreamingDqThread(dqEngines: DqEngines, dqEngines.loadData(dataSources) // generate rule steps - val ruleSteps = RuleAdaptorGroup.genConcreteRuleSteps(evaluateRuleParam) + val ruleSteps = RuleAdaptorGroup.genConcreteRuleSteps(evaluateRuleParam, StreamingProcessType) // run rules dqEngines.runRuleSteps(ruleSteps) val ct = new Date().getTime - persist.log(ct, s"calculation using time: ${ct - st} ms") + appPersist.log(ct, s"calculation using time: ${ct - st} ms") // persist results - dqEngines.persistAllResults(ruleSteps, persist) + dqEngines.persistAllResults(ruleSteps, appPersist) TimeInfoCache.endTimeInfoCache val et = new Date().getTime - persist.log(et, s"persist using time: ${et - ct} ms") + appPersist.log(et, s"persist using time: ${et - ct} ms") } catch { case e: Throwable => error(s"process error: ${e.getMessage}") diff --git a/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/DataFrameOprAdaptor.scala b/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/DataFrameOprAdaptor.scala index 00dfab9af..c7219233e 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/DataFrameOprAdaptor.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/DataFrameOprAdaptor.scala @@ -18,9 +18,10 @@ under the License. */ package org.apache.griffin.measure.rules.adaptor +import org.apache.griffin.measure.algo.ProcessType import org.apache.griffin.measure.rules.step._ -case class DataFrameOprAdaptor() extends RuleAdaptor { +case class DataFrameOprAdaptor(processType: ProcessType) extends RuleAdaptor { def genRuleStep(param: Map[String, Any]): Seq[RuleStep] = { DfOprStep(getName(param), getRule(param), getDetails(param), getPersistType(param)) :: Nil diff --git a/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/GriffinDslAdaptor.scala b/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/GriffinDslAdaptor.scala index 2ff7c816a..3148f97ad 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/GriffinDslAdaptor.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/GriffinDslAdaptor.scala @@ -18,13 +18,18 @@ under the License. */ package org.apache.griffin.measure.rules.adaptor +import org.apache.griffin.measure.algo.{BatchProcessType, ProcessType, StreamingProcessType} +import org.apache.griffin.measure.data.connector.GroupByColumn import org.apache.griffin.measure.rules.dsl._ import org.apache.griffin.measure.rules.dsl.analyzer._ import org.apache.griffin.measure.rules.dsl.expr._ import org.apache.griffin.measure.rules.dsl.parser.GriffinDslParser import org.apache.griffin.measure.rules.step._ -case class GriffinDslAdaptor(dataSourceNames: Seq[String], functionNames: Seq[String]) extends RuleAdaptor { +case class GriffinDslAdaptor(dataSourceNames: Seq[String], + functionNames: Seq[String], + processType: ProcessType + ) extends RuleAdaptor { object StepInfo { val _Name = "name" @@ -183,8 +188,13 @@ case class GriffinDslAdaptor(dataSourceNames: Seq[String], functionNames: Seq[St // 2. miss count val missTableName = "_miss_" val missColName = getNameOpt(details, AccuracyInfo._Miss).getOrElse(AccuracyInfo._Miss) - val missSql = { - s"SELECT COUNT(*) AS `${missColName}` FROM `${missRecordsName}`" + val missSql = processType match { + case BatchProcessType => { + s"SELECT COUNT(*) AS `${missColName}` FROM `${missRecordsName}`" + } + case StreamingProcessType => { + s"SELECT `${GroupByColumn.tmst}`, COUNT(*) AS `${missColName}` FROM `${missRecordsName}` GROUP BY `${GroupByColumn.tmst}`" + } } val missStep = SparkSqlStep( missTableName, @@ -196,8 +206,13 @@ case class GriffinDslAdaptor(dataSourceNames: Seq[String], functionNames: Seq[St // 3. total count val totalTableName = "_total_" val totalColName = getNameOpt(details, AccuracyInfo._Total).getOrElse(AccuracyInfo._Total) - val totalSql = { - s"SELECT COUNT(*) AS `${totalColName}` FROM `${sourceName}`" + val totalSql = processType match { + case BatchProcessType => { + s"SELECT COUNT(*) AS `${totalColName}` FROM `${sourceName}`" + } + case StreamingProcessType => { + s"SELECT `${GroupByColumn.tmst}`, COUNT(*) AS `${totalColName}` FROM `${sourceName}` GROUP BY `${GroupByColumn.tmst}`" + } } val totalStep = SparkSqlStep( totalTableName, @@ -208,19 +223,31 @@ case class GriffinDslAdaptor(dataSourceNames: Seq[String], functionNames: Seq[St // 4. accuracy metric val matchedColName = getNameOpt(details, AccuracyInfo._Matched).getOrElse(AccuracyInfo._Matched) - val accuracyMetricSql = { - s""" - |SELECT `${totalTableName}`.`${totalColName}` AS `${totalColName}`, - |`${missTableName}`.`${missColName}` AS `${missColName}`, - |(`${totalColName}` - `${missColName}`) AS `${matchedColName}` - |FROM `${totalTableName}` JOIN `${missTableName}` - """.stripMargin + val accuracyMetricSql = processType match { + case BatchProcessType => { + s""" + |SELECT `${totalTableName}`.`${totalColName}` AS `${totalColName}`, + |`${missTableName}`.`${missColName}` AS `${missColName}`, + |(`${totalColName}` - `${missColName}`) AS `${matchedColName}` + |FROM `${totalTableName}` JOIN `${missTableName}` + """.stripMargin + } + case StreamingProcessType => { + s""" + |SELECT `${missTableName}`.`${GroupByColumn.tmst}`, + |`${totalTableName}`.`${totalColName}` AS `${totalColName}`, + |`${missTableName}`.`${missColName}` AS `${missColName}`, + |(`${totalColName}` - `${missColName}`) AS `${matchedColName}` + |FROM `${totalTableName}` JOIN `${missTableName}` + |ON `${totalTableName}`.`${GroupByColumn.tmst}` = `${missTableName}`.`${GroupByColumn.tmst}` + """.stripMargin + } } val accuracyMetricName = resultName(details, AccuracyInfo._Accuracy) val accuracyMetricStep = SparkSqlStep( accuracyMetricName, accuracyMetricSql, - Map[String, Any](), + details, resultPersistType(details, AccuracyInfo._Accuracy, MetricPersistType) ) diff --git a/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/RuleAdaptor.scala b/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/RuleAdaptor.scala index 3641ea499..0c59258c8 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/RuleAdaptor.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/RuleAdaptor.scala @@ -20,6 +20,8 @@ package org.apache.griffin.measure.rules.adaptor import java.util.concurrent.atomic.AtomicLong +import org.apache.griffin.measure.algo._ + import scala.collection.mutable.{Set => MutableSet} import org.apache.griffin.measure.config.params.user._ import org.apache.griffin.measure.log.Loggable @@ -28,6 +30,8 @@ import org.apache.griffin.measure.rules.dsl.{DslType, PersistType} trait RuleAdaptor extends Loggable with Serializable { + val processType: ProcessType + val _name = "name" val _rule = "rule" val _persistType = "persist.type" diff --git a/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/RuleAdaptorGroup.scala b/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/RuleAdaptorGroup.scala index 1bf970f36..0e07c742e 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/RuleAdaptorGroup.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/RuleAdaptorGroup.scala @@ -18,6 +18,7 @@ under the License. */ package org.apache.griffin.measure.rules.adaptor +import org.apache.griffin.measure.algo.ProcessType import org.apache.griffin.measure.config.params.user._ import org.apache.griffin.measure.rules.dsl._ import org.apache.griffin.measure.rules.step._ @@ -46,11 +47,11 @@ object RuleAdaptorGroup { } } - private def genRuleAdaptor(dslType: DslType, dsNames: Seq[String]): Option[RuleAdaptor] = { + private def genRuleAdaptor(dslType: DslType, dsNames: Seq[String], procType: ProcessType): Option[RuleAdaptor] = { dslType match { - case SparkSqlType => Some(SparkSqlAdaptor()) - case DfOprType => Some(DataFrameOprAdaptor()) - case GriffinDslType => Some(GriffinDslAdaptor(dsNames, functionNames)) + case SparkSqlType => Some(SparkSqlAdaptor(procType)) + case DfOprType => Some(DataFrameOprAdaptor(procType)) + case GriffinDslType => Some(GriffinDslAdaptor(dsNames, functionNames, procType)) case _ => None } } @@ -71,18 +72,18 @@ object RuleAdaptorGroup { // steps // } - def genConcreteRuleSteps(evaluateRuleParam: EvaluateRuleParam): Seq[ConcreteRuleStep] = { + def genConcreteRuleSteps(evaluateRuleParam: EvaluateRuleParam, procType: ProcessType): Seq[ConcreteRuleStep] = { val dslTypeStr = if (evaluateRuleParam.dslType == null) "" else evaluateRuleParam.dslType val defaultDslType = DslType(dslTypeStr) val ruleParams = evaluateRuleParam.rules - genConcreteRuleSteps(ruleParams, defaultDslType) + genConcreteRuleSteps(ruleParams, defaultDslType, procType) } - def genConcreteRuleSteps(ruleParams: Seq[Map[String, Any]], defDslType: DslType): Seq[ConcreteRuleStep] = { + def genConcreteRuleSteps(ruleParams: Seq[Map[String, Any]], defDslType: DslType, procType: ProcessType): Seq[ConcreteRuleStep] = { val (steps, dsNames) = ruleParams.foldLeft((Seq[ConcreteRuleStep](), dataSourceNames)) { (res, param) => val (preSteps, preNames) = res val dslType = getDslType(param, defDslType) - val (curSteps, curNames) = genRuleAdaptor(dslType, preNames) match { + val (curSteps, curNames) = genRuleAdaptor(dslType, preNames, procType) match { case Some(ruleAdaptor) => (ruleAdaptor.genConcreteRuleStep(param), preNames ++ ruleAdaptor.getTempSourceNames(param)) case _ => (Nil, preNames) } diff --git a/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/SparkSqlAdaptor.scala b/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/SparkSqlAdaptor.scala index d6890093e..f4ec18720 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/SparkSqlAdaptor.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/SparkSqlAdaptor.scala @@ -18,10 +18,11 @@ under the License. */ package org.apache.griffin.measure.rules.adaptor +import org.apache.griffin.measure.algo.ProcessType import org.apache.griffin.measure.rules.dsl.PersistType import org.apache.griffin.measure.rules.step._ -case class SparkSqlAdaptor() extends RuleAdaptor { +case class SparkSqlAdaptor(processType: ProcessType) extends RuleAdaptor { def genRuleStep(param: Map[String, Any]): Seq[RuleStep] = { SparkSqlStep(getName(param), getRule(param), getDetails(param), getPersistType(param)) :: Nil diff --git a/measure/src/test/resources/config-test-accuracy-streaming.json b/measure/src/test/resources/config-test-accuracy-streaming.json index 95c3ee12b..5a2ec3c38 100644 --- a/measure/src/test/resources/config-test-accuracy-streaming.json +++ b/measure/src/test/resources/config-test-accuracy-streaming.json @@ -110,7 +110,8 @@ }, "miss": "miss_count", "total": "total_count", - "matched": "matched_count" + "matched": "matched_count", + "group.metric": true } } ] diff --git a/measure/src/test/resources/config-test-profiling.json b/measure/src/test/resources/config-test-profiling.json index 731c4dddc..bd8644e3b 100644 --- a/measure/src/test/resources/config-test-profiling.json +++ b/measure/src/test/resources/config-test-profiling.json @@ -32,6 +32,18 @@ }, "group.metric": true } + }, + { + "dsl.type": "griffin-dsl", + "dq.type": "profiling", + "rule": "source.user_id.count() as cnt", + "details": { + "source": "source", + "profiling": { + "name": "count", + "persist.type": "metric" + } + } } ] } diff --git a/measure/src/test/scala/org/apache/griffin/measure/process/BatchProcessTest.scala b/measure/src/test/scala/org/apache/griffin/measure/process/BatchProcessTest.scala index 94998da97..41e61c5ad 100644 --- a/measure/src/test/scala/org/apache/griffin/measure/process/BatchProcessTest.scala +++ b/measure/src/test/scala/org/apache/griffin/measure/process/BatchProcessTest.scala @@ -18,7 +18,7 @@ under the License. */ package org.apache.griffin.measure.process -import org.apache.griffin.measure.algo.ProcessType +import org.apache.griffin.measure.algo._ import org.apache.griffin.measure.config.params.env._ import org.apache.griffin.measure.config.params.user._ import org.apache.griffin.measure.config.params._ @@ -77,10 +77,10 @@ class BatchProcessTest extends FunSuite with Matchers with BeforeAndAfter with L } test ("batch process") { - val procType = allParam.userParam.procType + val procType = ProcessType(allParam.userParam.procType) val proc: DqProcess = procType match { - case ProcessType.batch() => BatchDqProcess(allParam) - case ProcessType.streaming() => StreamingDqProcess(allParam) + case BatchProcessType => BatchDqProcess(allParam) + case StreamingProcessType => StreamingDqProcess(allParam) case _ => { error(s"${procType} is unsupported process type!") sys.exit(-4) diff --git a/measure/src/test/scala/org/apache/griffin/measure/process/JsonParseTest.scala b/measure/src/test/scala/org/apache/griffin/measure/process/JsonParseTest.scala index 11a9a1550..04683453a 100644 --- a/measure/src/test/scala/org/apache/griffin/measure/process/JsonParseTest.scala +++ b/measure/src/test/scala/org/apache/griffin/measure/process/JsonParseTest.scala @@ -18,7 +18,7 @@ under the License. */ package org.apache.griffin.measure.process -import org.apache.griffin.measure.algo.ProcessType +import org.apache.griffin.measure.algo._ import org.apache.griffin.measure.config.params._ import org.apache.griffin.measure.config.params.env._ import org.apache.griffin.measure.config.params.user._ diff --git a/measure/src/test/scala/org/apache/griffin/measure/process/StreamingProcessTest.scala b/measure/src/test/scala/org/apache/griffin/measure/process/StreamingProcessTest.scala index cbbcedf70..5e4424f17 100644 --- a/measure/src/test/scala/org/apache/griffin/measure/process/StreamingProcessTest.scala +++ b/measure/src/test/scala/org/apache/griffin/measure/process/StreamingProcessTest.scala @@ -18,7 +18,7 @@ under the License. */ package org.apache.griffin.measure.process -import org.apache.griffin.measure.algo.ProcessType +import org.apache.griffin.measure.algo._ import org.apache.griffin.measure.config.params._ import org.apache.griffin.measure.config.params.env._ import org.apache.griffin.measure.config.params.user._ @@ -77,10 +77,10 @@ class StreamingProcessTest extends FunSuite with Matchers with BeforeAndAfter wi } test ("streaming process") { - val procType = allParam.userParam.procType + val procType = ProcessType(allParam.userParam.procType) val proc: DqProcess = procType match { - case ProcessType.batch() => BatchDqProcess(allParam) - case ProcessType.streaming() => StreamingDqProcess(allParam) + case BatchProcessType => BatchDqProcess(allParam) + case StreamingProcessType => StreamingDqProcess(allParam) case _ => { error(s"${procType} is unsupported process type!") sys.exit(-4) From 1c102458fb36094674757197c92adb3f73b9d0a7 Mon Sep 17 00:00:00 2001 From: Lionel Liu Date: Wed, 13 Sep 2017 09:32:12 +0800 Subject: [PATCH 066/111] not persisted properly --- .../org/apache/griffin/measure/process/BatchDqProcess.scala | 3 ++- .../apache/griffin/measure/process/StreamingDqProcess.scala | 5 +++-- .../apache/griffin/measure/process/StreamingDqThread.scala | 3 ++- 3 files changed, 7 insertions(+), 4 deletions(-) diff --git a/measure/src/main/scala/org/apache/griffin/measure/process/BatchDqProcess.scala b/measure/src/main/scala/org/apache/griffin/measure/process/BatchDqProcess.scala index 356a50436..2a7853cff 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/process/BatchDqProcess.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/process/BatchDqProcess.scala @@ -69,7 +69,8 @@ case class BatchDqProcess(allParam: AllParam) extends DqProcess { val startTime = new Date().getTime() // get persists to persist measure result - val persist: Persist = PersistFactory(envParam.persistParams, metricName).getPersists(startTime) + val persistFactory = PersistFactory(envParam.persistParams, metricName) + val persist: Persist = persistFactory.getPersists(startTime) // persist start id val applicationId = sparkContext.applicationId diff --git a/measure/src/main/scala/org/apache/griffin/measure/process/StreamingDqProcess.scala b/measure/src/main/scala/org/apache/griffin/measure/process/StreamingDqProcess.scala index 428a44b54..ddf76bdae 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/process/StreamingDqProcess.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/process/StreamingDqProcess.scala @@ -86,7 +86,8 @@ case class StreamingDqProcess(allParam: AllParam) extends DqProcess { val startTime = new Date().getTime() // get persists to persist measure result - val persist: Persist = PersistFactory(envParam.persistParams, metricName).getPersists(startTime) + val persistFactory = PersistFactory(envParam.persistParams, metricName) + val persist: Persist = persistFactory.getPersists(startTime) // persist start id val applicationId = sparkContext.applicationId @@ -100,7 +101,7 @@ case class StreamingDqProcess(allParam: AllParam) extends DqProcess { dataSources.foreach(_.init) // process thread - val dqThread = StreamingDqThread(dqEngines, dataSources, userParam.evaluateRuleParam, persist) + val dqThread = StreamingDqThread(dqEngines, dataSources, userParam.evaluateRuleParam, persistFactory, persist) // init data sources // dqEngines.loadData(dataSources) diff --git a/measure/src/main/scala/org/apache/griffin/measure/process/StreamingDqThread.scala b/measure/src/main/scala/org/apache/griffin/measure/process/StreamingDqThread.scala index 5a89bfb5b..1a8cfed36 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/process/StreamingDqThread.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/process/StreamingDqThread.scala @@ -26,13 +26,14 @@ import org.apache.griffin.measure.cache.info.{InfoCacheInstance, TimeInfoCache} import org.apache.griffin.measure.config.params.user.EvaluateRuleParam import org.apache.griffin.measure.data.source.DataSource import org.apache.griffin.measure.log.Loggable -import org.apache.griffin.measure.persist.Persist +import org.apache.griffin.measure.persist.{Persist, PersistFactory} import org.apache.griffin.measure.process.engine.DqEngines import org.apache.griffin.measure.rules.adaptor.RuleAdaptorGroup case class StreamingDqThread(dqEngines: DqEngines, dataSources: Seq[DataSource], evaluateRuleParam: EvaluateRuleParam, + persistFactory: PersistFactory, appPersist: Persist ) extends Runnable with Loggable { From 23c320e6c32a937c5b405101b6f2615ed6ea1ac3 Mon Sep 17 00:00:00 2001 From: Lionel Liu Date: Wed, 13 Sep 2017 18:14:34 +0800 Subject: [PATCH 067/111] change --- .../measure/cache/metric/CacheMetric.scala | 27 +++ .../cache/metric/CacheMetricProcesser.scala | 72 +++++++ .../data/connector/DataConnector.scala | 2 +- .../batch/AvroBatchDataConnector.scala | 6 +- .../batch/HiveBatchDataConnector.scala | 6 +- .../streaming/StreamingDataConnector.scala | 2 +- .../measure/data/source/DataSource.scala | 8 +- .../griffin/measure/metric/Metric.scala | 23 +++ .../measure/metric/MetricCompare.scala | 29 +++ .../measure/process/BatchDqProcess.scala | 4 +- .../measure/process/StreamingDqThread.scala | 4 +- .../process/engine/DataFrameOprEngine.scala | 191 +++++++++++++----- .../measure/process/engine/DqEngine.scala | 6 +- .../measure/process/engine/DqEngines.scala | 53 +++-- .../process/engine/SparkSqlEngine.scala | 121 +++++++---- .../rules/adaptor/GriffinDslAdaptor.scala | 143 ++++++++----- .../rules/adaptor/SparkSqlAdaptor.scala | 16 +- .../rules/preproc/PreProcRuleGenerator.scala | 16 +- .../measure/rules/step/ConcreteRuleStep.scala | 14 +- .../config-test-accuracy-streaming.json | 3 +- .../test/resources/config-test-profiling.json | 13 -- 21 files changed, 564 insertions(+), 195 deletions(-) create mode 100644 measure/src/main/scala/org/apache/griffin/measure/cache/metric/CacheMetric.scala create mode 100644 measure/src/main/scala/org/apache/griffin/measure/cache/metric/CacheMetricProcesser.scala create mode 100644 measure/src/main/scala/org/apache/griffin/measure/metric/Metric.scala create mode 100644 measure/src/main/scala/org/apache/griffin/measure/metric/MetricCompare.scala diff --git a/measure/src/main/scala/org/apache/griffin/measure/cache/metric/CacheMetric.scala b/measure/src/main/scala/org/apache/griffin/measure/cache/metric/CacheMetric.scala new file mode 100644 index 000000000..b4a63a309 --- /dev/null +++ b/measure/src/main/scala/org/apache/griffin/measure/cache/metric/CacheMetric.scala @@ -0,0 +1,27 @@ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ +package org.apache.griffin.measure.cache.metric + +case class CacheMetric(timeGroup: Long, updateTime: Long, metric: Map[String, Any]) { + + def olderThan(ut: Long): Boolean = { + updateTime < ut + } + +} diff --git a/measure/src/main/scala/org/apache/griffin/measure/cache/metric/CacheMetricProcesser.scala b/measure/src/main/scala/org/apache/griffin/measure/cache/metric/CacheMetricProcesser.scala new file mode 100644 index 000000000..7da51dbf7 --- /dev/null +++ b/measure/src/main/scala/org/apache/griffin/measure/cache/metric/CacheMetricProcesser.scala @@ -0,0 +1,72 @@ +///* +//Licensed to the Apache Software Foundation (ASF) under one +//or more contributor license agreements. See the NOTICE file +//distributed with this work for additional information +//regarding copyright ownership. The ASF licenses this file +//to you under the Apache License, Version 2.0 (the +//"License"); you may not use this file except in compliance +//with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +//Unless required by applicable law or agreed to in writing, +//software distributed under the License is distributed on an +//"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +//KIND, either express or implied. See the License for the +//specific language governing permissions and limitations +//under the License. +//*/ +//package org.apache.griffin.measure.cache.metric +// +//import org.apache.griffin.measure.log.Loggable +//import org.apache.griffin.measure.result._ +// +//import scala.collection.mutable.{Map => MutableMap} +// +//case class CacheMetricProcesser() extends Loggable { +// +// val cacheGroup: MutableMap[Long, CacheMetric] = MutableMap() +// +// def genUpdateCacheMetric(timeGroup: Long, updateTime: Long, metric: Map[String, Any]): Option[CacheMetric] = { +// cacheGroup.get(timeGroup) match { +// case Some(cr) => { +// if (cr.olderThan(updateTime)) { +// val existMetric = cr.metric +// val newMetric = existMetric +// val newMetric = existMetric.update(result.asInstanceOf[existResult.T]) +// if (existResult.differsFrom(newResult)) { +// Some(CacheMetric(timeGroup, updateTime, newResult)) +// } else None +// } else None +// } +// case _ => { +// Some(CacheMetric(timeGroup, updateTime, result)) +// } +// } +// } +// +// def update(cr: CacheMetric): Unit = { +// val t = cr.timeGroup +// cacheGroup.get(t) match { +// case Some(c) => { +// if (c.olderThan(cr.updateTime)) cacheGroup += (t -> cr) +// } +// case _ => cacheGroup += (t -> cr) +// } +// } +// +// def getCacheResult(timeGroup: Long): Option[CacheMetric] = { +// cacheGroup.get(timeGroup) +// } +// +// def refresh(overtime: Long): Unit = { +// val curCacheGroup = cacheGroup.toMap +// val deadCache = curCacheGroup.filter { pr => +// val (_, cr) = pr +// cr.timeGroup < overtime || cr.result.eventual() +// } +// info(s"=== dead cache group count: ${deadCache.size} ===") +// deadCache.keySet.foreach(cacheGroup -= _) +// } +// +//} diff --git a/measure/src/main/scala/org/apache/griffin/measure/data/connector/DataConnector.scala b/measure/src/main/scala/org/apache/griffin/measure/data/connector/DataConnector.scala index acb52fd46..6beaf18d0 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/data/connector/DataConnector.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/data/connector/DataConnector.scala @@ -38,7 +38,7 @@ trait DataConnector extends Loggable with Serializable { def init(): Unit - def data(): Option[DataFrame] + def data(ms: Long): Option[DataFrame] val dqEngines: DqEngines diff --git a/measure/src/main/scala/org/apache/griffin/measure/data/connector/batch/AvroBatchDataConnector.scala b/measure/src/main/scala/org/apache/griffin/measure/data/connector/batch/AvroBatchDataConnector.scala index e603b903a..d9bef1227 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/data/connector/batch/AvroBatchDataConnector.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/data/connector/batch/AvroBatchDataConnector.scala @@ -51,10 +51,12 @@ case class AvroBatchDataConnector(sqlContext: SQLContext, dqEngines: DqEngines, HdfsUtil.existPath(concreteFileFullPath) } - def data(): Option[DataFrame] = { + def data(ms: Long): Option[DataFrame] = { try { val df = sqlContext.read.format("com.databricks.spark.avro").load(concreteFileFullPath) - Some(df) + val dfOpt = Some(df) + val preDfOpt = preProcess(dfOpt, ms) + preDfOpt } catch { case e: Throwable => { error(s"load avro file ${concreteFileFullPath} fails") diff --git a/measure/src/main/scala/org/apache/griffin/measure/data/connector/batch/HiveBatchDataConnector.scala b/measure/src/main/scala/org/apache/griffin/measure/data/connector/batch/HiveBatchDataConnector.scala index 81d7fe9b4..652d2b243 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/data/connector/batch/HiveBatchDataConnector.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/data/connector/batch/HiveBatchDataConnector.scala @@ -50,10 +50,12 @@ case class HiveBatchDataConnector(sqlContext: SQLContext, dqEngines: DqEngines, val concreteTableName = s"${database}.${tableName}" val partitions = partitionsString.split(";").map(s => s.split(",").map(_.trim)) - def data(): Option[DataFrame] = { + def data(ms: Long): Option[DataFrame] = { try { val df = sqlContext.sql(dataSql) - Some(df) + val dfOpt = Some(df) + val preDfOpt = preProcess(dfOpt, ms) + preDfOpt } catch { case e: Throwable => { error(s"load hive table ${concreteTableName} fails") diff --git a/measure/src/main/scala/org/apache/griffin/measure/data/connector/streaming/StreamingDataConnector.scala b/measure/src/main/scala/org/apache/griffin/measure/data/connector/streaming/StreamingDataConnector.scala index d9bb1f1d0..cc2176155 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/data/connector/streaming/StreamingDataConnector.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/data/connector/streaming/StreamingDataConnector.scala @@ -36,7 +36,7 @@ trait StreamingDataConnector extends DataConnector { def transform(rdd: RDD[(K, V)]): Option[DataFrame] - def data(): Option[DataFrame] = None + def data(ms: Long): Option[DataFrame] = None var dataSourceCacheOpt: Option[DataSourceCache] = None diff --git a/measure/src/main/scala/org/apache/griffin/measure/data/source/DataSource.scala b/measure/src/main/scala/org/apache/griffin/measure/data/source/DataSource.scala index 739676f06..921690615 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/data/source/DataSource.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/data/source/DataSource.scala @@ -38,8 +38,8 @@ case class DataSource(name: String, dataConnectors.foreach(_.init) } - def loadData(): Unit = { - data match { + def loadData(ms: Long): Unit = { + data(ms) match { case Some(df) => { df.registerTempTable(name) } @@ -49,9 +49,9 @@ case class DataSource(name: String, } } - private def data(): Option[DataFrame] = { + private def data(ms: Long): Option[DataFrame] = { val batchDataFrameOpt = batchDataConnectors.flatMap { dc => - dc.data + dc.data(ms) }.reduceOption(_ unionAll _) val cacheDataFrameOpt = dataSourceCacheOpt.flatMap(_.readData()) diff --git a/measure/src/main/scala/org/apache/griffin/measure/metric/Metric.scala b/measure/src/main/scala/org/apache/griffin/measure/metric/Metric.scala new file mode 100644 index 000000000..d52991301 --- /dev/null +++ b/measure/src/main/scala/org/apache/griffin/measure/metric/Metric.scala @@ -0,0 +1,23 @@ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ +package org.apache.griffin.measure.metric + +case class Metric(value: Map[String, Any]) extends Serializable { + +} diff --git a/measure/src/main/scala/org/apache/griffin/measure/metric/MetricCompare.scala b/measure/src/main/scala/org/apache/griffin/measure/metric/MetricCompare.scala new file mode 100644 index 000000000..1e100eff5 --- /dev/null +++ b/measure/src/main/scala/org/apache/griffin/measure/metric/MetricCompare.scala @@ -0,0 +1,29 @@ +///* +//Licensed to the Apache Software Foundation (ASF) under one +//or more contributor license agreements. See the NOTICE file +//distributed with this work for additional information +//regarding copyright ownership. The ASF licenses this file +//to you under the Apache License, Version 2.0 (the +//"License"); you may not use this file except in compliance +//with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +//Unless required by applicable law or agreed to in writing, +//software distributed under the License is distributed on an +//"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +//KIND, either express or implied. See the License for the +//specific language governing permissions and limitations +//under the License. +//*/ +//package org.apache.griffin.measure.metric +// +//case class MetricCompare(config: Map[String, String]) extends Serializable { +// +// val _update +// +// def differ(opt1: Option[Metric], opt2: Option[Metric]): Boolean = { +// +// } +// +//} diff --git a/measure/src/main/scala/org/apache/griffin/measure/process/BatchDqProcess.scala b/measure/src/main/scala/org/apache/griffin/measure/process/BatchDqProcess.scala index 2a7853cff..ba0764ff8 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/process/BatchDqProcess.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/process/BatchDqProcess.scala @@ -84,7 +84,7 @@ case class BatchDqProcess(allParam: AllParam) extends DqProcess { dataSources.foreach(_.init) // init data sources - dqEngines.loadData(dataSources) + dqEngines.loadData(dataSources, startTime) // generate rule steps val ruleSteps = RuleAdaptorGroup.genConcreteRuleSteps(userParam.evaluateRuleParam, BatchProcessType) @@ -93,7 +93,7 @@ case class BatchDqProcess(allParam: AllParam) extends DqProcess { dqEngines.runRuleSteps(ruleSteps) // persist results - dqEngines.persistAllResults(ruleSteps, persist) + dqEngines.persistAllResults(ruleSteps, persistFactory) // end time val endTime = new Date().getTime diff --git a/measure/src/main/scala/org/apache/griffin/measure/process/StreamingDqThread.scala b/measure/src/main/scala/org/apache/griffin/measure/process/StreamingDqThread.scala index 1a8cfed36..97bc619b3 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/process/StreamingDqThread.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/process/StreamingDqThread.scala @@ -53,7 +53,7 @@ case class StreamingDqThread(dqEngines: DqEngines, TimeInfoCache.startTimeInfoCache // init data sources - dqEngines.loadData(dataSources) + dqEngines.loadData(dataSources, st) // generate rule steps val ruleSteps = RuleAdaptorGroup.genConcreteRuleSteps(evaluateRuleParam, StreamingProcessType) @@ -65,7 +65,7 @@ case class StreamingDqThread(dqEngines: DqEngines, appPersist.log(ct, s"calculation using time: ${ct - st} ms") // persist results - dqEngines.persistAllResults(ruleSteps, appPersist) + dqEngines.persistAllResults(ruleSteps, persistFactory) TimeInfoCache.endTimeInfoCache diff --git a/measure/src/main/scala/org/apache/griffin/measure/process/engine/DataFrameOprEngine.scala b/measure/src/main/scala/org/apache/griffin/measure/process/engine/DataFrameOprEngine.scala index a2e9fd5a4..81578c5f8 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/process/engine/DataFrameOprEngine.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/process/engine/DataFrameOprEngine.scala @@ -20,13 +20,18 @@ package org.apache.griffin.measure.process.engine import java.util.Date +import org.apache.griffin.measure.cache.result.CacheResultProcesser import org.apache.griffin.measure.config.params.user.DataSourceParam +import org.apache.griffin.measure.data.connector.GroupByColumn import org.apache.griffin.measure.data.source.{DataSource, DataSourceFactory} -import org.apache.griffin.measure.persist.Persist +import org.apache.griffin.measure.persist.{Persist, PersistFactory} +import org.apache.griffin.measure.result.AccuracyResult import org.apache.griffin.measure.rules.dsl._ import org.apache.griffin.measure.rules.step._ import org.apache.griffin.measure.utils.JsonUtil -import org.apache.spark.sql.{DataFrame, SQLContext} +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.types.{LongType, StringType, StructField, StructType} +import org.apache.spark.sql.{DataFrame, Row, SQLContext} import org.apache.spark.streaming.StreamingContext case class DataFrameOprEngine(sqlContext: SQLContext, @transient ssc: StreamingContext @@ -45,6 +50,10 @@ case class DataFrameOprEngine(sqlContext: SQLContext, @transient ssc: StreamingC val df = DataFrameOprs.fromJson(sqlContext, details) df.registerTempTable(name) } + case DataFrameOprs._accuracy => { + val df = DataFrameOprs.accuracy(sqlContext, details) + df.registerTempTable(name) + } case _ => { throw new Exception(s"df opr [ ${rule} ] not supported") } @@ -61,32 +70,33 @@ case class DataFrameOprEngine(sqlContext: SQLContext, @transient ssc: StreamingC } } - def persistRecords(ruleStep: ConcreteRuleStep, persist: Persist): Boolean = { - val curTime = new Date().getTime - ruleStep match { - case DfOprStep(name, _, _, RecordPersistType) => { - try { - val pdf = sqlContext.table(s"`${name}`") - val records = pdf.toJSON - - persist.persistRecords(records, name) - - val recordLog = s"[ ${name} ] persist records" - persist.log(curTime, recordLog) - - true - } catch { - case e: Throwable => { - error(s"persist result ${name} error: ${e.getMessage}") - false - } - } - } - case _ => false - } + def persistRecords(ruleStep: ConcreteRuleStep, persistFactory: PersistFactory): Boolean = { +// val curTime = new Date().getTime +// ruleStep match { +// case DfOprStep(name, _, _, RecordPersistType) => { +// try { +// val pdf = sqlContext.table(s"`${name}`") +// val records = pdf.toJSON +// +// persist.persistRecords(records, name) +// +// val recordLog = s"[ ${name} ] persist records" +// persist.log(curTime, recordLog) +// +// true +// } catch { +// case e: Throwable => { +// error(s"persist result ${name} error: ${e.getMessage}") +// false +// } +// } +// } +// case _ => false +// } + true } - def collectMetrics(ruleStep: ConcreteRuleStep): Map[String, Any] = { + def collectMetrics(ruleStep: ConcreteRuleStep): Map[Long, Map[String, Any]] = { val emptyMap = Map[String, Any]() ruleStep match { case DfOprStep(name, _, _, MetricPersistType) => { @@ -94,35 +104,66 @@ case class DataFrameOprEngine(sqlContext: SQLContext, @transient ssc: StreamingC val pdf = sqlContext.table(s"`${name}`") val records = pdf.toJSON.collect() - if (ruleStep.isGroupMetric) { - val arr = records.flatMap { rec => - try { - Some(JsonUtil.toAnyMap(rec)) - } catch { - case e: Throwable => None - } - } - Map[String, Any]((name -> arr)) - } else { - records.headOption match { - case Some(head) => { - try { - JsonUtil.toAnyMap(head) - } catch { - case e: Throwable => emptyMap + val pairs = records.flatMap { rec => + try { + val value = JsonUtil.toAnyMap(rec) + value.get(GroupByColumn.tmst) match { + case Some(t) => { + val key = t.toString.toLong + Some((key, value)) } + case _ => None } - case _ => emptyMap + } catch { + case e: Throwable => None + } + } + val groupedPairs = pairs.foldLeft(Map[Long, Seq[Map[String, Any]]]()) { (ret, pair) => + val (k, v) = pair + ret.get(k) match { + case Some(seq) => ret + (k -> (seq :+ v)) + case _ => ret + (k -> (v :: Nil)) } } + groupedPairs.mapValues { vs => + if (vs.size > 1) { + Map[String, Any]((name -> vs)) + } else { + vs.headOption.getOrElse(emptyMap) + } + } + +// if (ruleStep.isGroupMetric) { +// val arr = records.flatMap { rec => +// try { +// Some(JsonUtil.toAnyMap(rec)) +// } catch { +// case e: Throwable => None +// } +// } +// Map[String, Any]((name -> arr)) +// } else { +// records.headOption match { +// case Some(head) => { +// try { +// JsonUtil.toAnyMap(head) +// } catch { +// case e: Throwable => emptyMap +// } +// } +// case _ => emptyMap +// } +// } } catch { case e: Throwable => { error(s"persist result ${name} error: ${e.getMessage}") - emptyMap +// emptyMap + Map[Long, Map[String, Any]]() } } } - case _ => emptyMap +// case _ => emptyMap + case _ => Map[Long, Map[String, Any]]() } } @@ -151,6 +192,7 @@ case class DataFrameOprEngine(sqlContext: SQLContext, @transient ssc: StreamingC object DataFrameOprs { final val _fromJson = "from_json" + final val _accuracy = "accuracy" def fromJson(sqlContext: SQLContext, details: Map[String, Any]): DataFrame = { val _dfName = "df.name" @@ -166,6 +208,65 @@ object DataFrameOprs { sqlContext.read.json(rdd) } + final val cacheResultProcesser = CacheResultProcesser() + + def accuracy(sqlContext: SQLContext, details: Map[String, Any]): DataFrame = { + val _dfName = "df.name" + val _miss = "miss" + val _total = "total" + val _matched = "matched" + val _tmst = "tmst" + val dfName = details.getOrElse(_dfName, _dfName).toString + val miss = details.getOrElse(_miss, _miss).toString + val total = details.getOrElse(_total, _total).toString + val matched = details.getOrElse(_matched, _matched).toString + val tmst = details.getOrElse(_tmst, _tmst).toString + + val updateTime = new Date().getTime + + def getLong(r: Row, k: String): Long = { + try { + r.getAs[Long](k) + } catch { + case e: Throwable => 0L + } + } + + val df = sqlContext.table(s"`${dfName}`") + val results = df.map { row => + val t = getLong(row, tmst) + val missCount = getLong(row, miss) + val totalCount = getLong(row, total) + val ar = AccuracyResult(missCount, totalCount) + (t, ar) + }.collect + + val updateResults = results.flatMap { pair => + val (t, result) = pair + val updatedCacheResultOpt = cacheResultProcesser.genUpdateCacheResult(t, updateTime, result) + updatedCacheResultOpt + } + + // update + updateResults.foreach { r => + cacheResultProcesser.update(r) + } + + val schema = StructType(Array( + StructField(tmst, LongType), + StructField(miss, LongType), + StructField(total, LongType), + StructField(matched, LongType) + )) + val rows = updateResults.map { r => + val ar = r.result.asInstanceOf[AccuracyResult] + Row(r.timeGroup, ar.miss, ar.total, ar.getMatch) + } + val rowRdd = sqlContext.sparkContext.parallelize(rows) + sqlContext.createDataFrame(rowRdd, schema) + + } + } diff --git a/measure/src/main/scala/org/apache/griffin/measure/process/engine/DqEngine.scala b/measure/src/main/scala/org/apache/griffin/measure/process/engine/DqEngine.scala index c2507def8..c90240704 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/process/engine/DqEngine.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/process/engine/DqEngine.scala @@ -21,7 +21,7 @@ package org.apache.griffin.measure.process.engine import org.apache.griffin.measure.config.params.user.DataSourceParam import org.apache.griffin.measure.data.source.DataSource import org.apache.griffin.measure.log.Loggable -import org.apache.griffin.measure.persist.Persist +import org.apache.griffin.measure.persist.{Persist, PersistFactory} import org.apache.griffin.measure.rules.dsl._ import org.apache.griffin.measure.rules.step._ import org.apache.spark.sql.DataFrame @@ -34,7 +34,7 @@ trait DqEngine extends Loggable with Serializable { // def persistResults(ruleSteps: Seq[ConcreteRuleStep], persist: Persist, persistType: PersistType): Boolean - def persistRecords(ruleStep: ConcreteRuleStep, persist: Persist): Boolean - def collectMetrics(ruleStep: ConcreteRuleStep): Map[String, Any] + def persistRecords(ruleStep: ConcreteRuleStep, persistFactory: PersistFactory): Boolean + def collectMetrics(ruleStep: ConcreteRuleStep): Map[Long, Map[String, Any]] } diff --git a/measure/src/main/scala/org/apache/griffin/measure/process/engine/DqEngines.scala b/measure/src/main/scala/org/apache/griffin/measure/process/engine/DqEngines.scala index 7b0b34fa2..5da323724 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/process/engine/DqEngines.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/process/engine/DqEngines.scala @@ -19,9 +19,10 @@ under the License. package org.apache.griffin.measure.process.engine import org.apache.griffin.measure.config.params.user.DataSourceParam +import org.apache.griffin.measure.data.connector.GroupByColumn import org.apache.griffin.measure.data.source._ import org.apache.griffin.measure.log.Loggable -import org.apache.griffin.measure.persist.Persist +import org.apache.griffin.measure.persist.{Persist, PersistFactory} import org.apache.griffin.measure.rules.dsl._ import org.apache.griffin.measure.rules.step._ @@ -29,12 +30,12 @@ case class DqEngines(engines: Seq[DqEngine]) extends DqEngine { val persistOrder: List[PersistType] = List(MetricPersistType, RecordPersistType) - def loadData(dataSources: Seq[DataSource]): Unit = { + def loadData(dataSources: Seq[DataSource], ms: Long): Unit = { // val dataSources = dataSourceParams.flatMap { param => // genDataSource(param) // } dataSources.foreach { ds => - ds.loadData + ds.loadData(ms) } } @@ -44,7 +45,7 @@ case class DqEngines(engines: Seq[DqEngine]) extends DqEngine { } } - def persistAllResults(ruleSteps: Seq[ConcreteRuleStep], persist: Persist): Unit = { + def persistAllResults(ruleSteps: Seq[ConcreteRuleStep], persistFactory: PersistFactory): Unit = { // 1. group by same persist types val groupedRuleSteps = ruleSteps.groupBy(_.persistType) @@ -56,12 +57,28 @@ case class DqEngines(engines: Seq[DqEngine]) extends DqEngine { } prstType match { case MetricPersistType => { - val metrics = steps.foldLeft(Map[String, Any]())(_ ++ collectMetrics(_)) - persist.persistMetrics(metrics) +// val metrics = steps.foldLeft(Map[String, Any]())(_ ++ collectMetrics(_)) + val metrics: Map[Long, Map[String, Any]] = { + steps.foldLeft(Map[Long, Map[String, Any]]()) { (ret, step) => + val metrics = collectMetrics(step) + metrics.foldLeft(ret) { (total, pair) => + val (k, v) = pair + ret.get(k) match { + case Some(map) => ret + (k -> (map ++ v)) + case _ => ret + pair + } + } + } + } + metrics.foreach { pair => + val (t, metric) = pair + val persist = persistFactory.getPersists(t) + persist.persistMetrics(metric) + } } case RecordPersistType => { steps.foreach { ruleStep => - persistRecords(ruleStep, persist) + persistRecords(ruleStep, persistFactory) } } case _ => { @@ -87,19 +104,31 @@ case class DqEngines(engines: Seq[DqEngine]) extends DqEngine { ret } - def persistRecords(ruleStep: ConcreteRuleStep, persist: Persist): Boolean = { + def persistRecords(ruleStep: ConcreteRuleStep, persistFactory: PersistFactory): Boolean = { val ret = engines.foldLeft(false) { (done, engine) => - done || engine.persistRecords(ruleStep, persist) + done || engine.persistRecords(ruleStep, persistFactory) } if (!ret) error(s"persist result warn: no dq engine support ${ruleStep}") ret } - def collectMetrics(ruleStep: ConcreteRuleStep): Map[String, Any] = { - val ret = engines.foldLeft(Map[String, Any]()) { (ret, engine) => + def collectMetrics(ruleStep: ConcreteRuleStep): Map[Long, Map[String, Any]] = { + val ret = engines.foldLeft(Map[Long, Map[String, Any]]()) { (ret, engine) => ret ++ engine.collectMetrics(ruleStep) } - if (ret.isEmpty) error(s"collect metrics warn: no metrics collected for ${ruleStep}") + if (ret.isEmpty) warn(s"collect metrics warn: no metrics collected for ${ruleStep}") ret +// val ret = engines.foldLeft(Map[Long, Map[String, Any]]()) { (ret, engine) => +// val metrics: Map[Long, Map[String, Any]] = engine.collectMetrics(ruleStep) +// metrics.foldLeft(ret) { (total, pair) => +// val (k, v) = pair +// ret.get(k) match { +// case Some(map) => ret + (k -> (map ++ v)) +// case _ => ret + pair +// } +// } +// } +// if (ret.isEmpty) error(s"collect metrics warn: no metrics collected for ${ruleStep}") +// ret } // def persistResults(ruleSteps: Seq[ConcreteRuleStep], persist: Persist, persistType: PersistType): Boolean = { diff --git a/measure/src/main/scala/org/apache/griffin/measure/process/engine/SparkSqlEngine.scala b/measure/src/main/scala/org/apache/griffin/measure/process/engine/SparkSqlEngine.scala index 0b95a35dd..ec77fe268 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/process/engine/SparkSqlEngine.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/process/engine/SparkSqlEngine.scala @@ -21,8 +21,9 @@ package org.apache.griffin.measure.process.engine import java.util.Date import org.apache.griffin.measure.config.params.user.DataSourceParam +import org.apache.griffin.measure.data.connector.GroupByColumn import org.apache.griffin.measure.data.source._ -import org.apache.griffin.measure.persist.Persist +import org.apache.griffin.measure.persist.{Persist, PersistFactory} import org.apache.griffin.measure.rules.dsl._ import org.apache.griffin.measure.rules.step._ import org.apache.griffin.measure.utils.JsonUtil @@ -54,32 +55,33 @@ case class SparkSqlEngine(sqlContext: SQLContext, @transient ssc: StreamingConte } } - def persistRecords(ruleStep: ConcreteRuleStep, persist: Persist): Boolean = { - val curTime = new Date().getTime - ruleStep match { - case SparkSqlStep(name, _, _, RecordPersistType) => { - try { - val pdf = sqlContext.table(s"`${name}`") - val records = pdf.toJSON - - persist.persistRecords(records, name) - - val recordLog = s"[ ${name} ] persist records" - persist.log(curTime, recordLog) - - true - } catch { - case e: Throwable => { - error(s"persist result ${name} error: ${e.getMessage}") - false - } - } - } - case _ => false - } + def persistRecords(ruleStep: ConcreteRuleStep, persistFactory: PersistFactory): Boolean = { +// val curTime = new Date().getTime +// ruleStep match { +// case SparkSqlStep(name, _, _, RecordPersistType) => { +// try { +// val pdf = sqlContext.table(s"`${name}`") +// val records = pdf.toJSON +// +// persist.persistRecords(records, name) +// +// val recordLog = s"[ ${name} ] persist records" +// persist.log(curTime, recordLog) +// +// true +// } catch { +// case e: Throwable => { +// error(s"persist result ${name} error: ${e.getMessage}") +// false +// } +// } +// } +// case _ => false +// } + true } - def collectMetrics(ruleStep: ConcreteRuleStep): Map[String, Any] = { + def collectMetrics(ruleStep: ConcreteRuleStep): Map[Long, Map[String, Any]] = { val emptyMap = Map[String, Any]() ruleStep match { case SparkSqlStep(name, _, _, MetricPersistType) => { @@ -87,35 +89,66 @@ case class SparkSqlEngine(sqlContext: SQLContext, @transient ssc: StreamingConte val pdf = sqlContext.table(s"`${name}`") val records = pdf.toJSON.collect() - if (ruleStep.isGroupMetric) { - val arr = records.flatMap { rec => - try { - Some(JsonUtil.toAnyMap(rec)) - } catch { - case e: Throwable => None - } - } - Map[String, Any]((name -> arr)) - } else { - records.headOption match { - case Some(head) => { - try { - JsonUtil.toAnyMap(head) - } catch { - case e: Throwable => emptyMap + val pairs = records.flatMap { rec => + try { + val value = JsonUtil.toAnyMap(rec) + value.get(GroupByColumn.tmst) match { + case Some(t) => { + val key = t.toString.toLong + Some((key, value)) } + case _ => None } - case _ => emptyMap + } catch { + case e: Throwable => None + } + } + val groupedPairs = pairs.foldLeft(Map[Long, Seq[Map[String, Any]]]()) { (ret, pair) => + val (k, v) = pair + ret.get(k) match { + case Some(seq) => ret + (k -> (seq :+ v)) + case _ => ret + (k -> (v :: Nil)) + } + } + groupedPairs.mapValues { vs => + if (vs.size > 1) { + Map[String, Any]((name -> vs)) + } else { + vs.headOption.getOrElse(emptyMap) } } + +// if (ruleStep.isGroupMetric) { +// val arr = records.flatMap { rec => +// try { +// Some(JsonUtil.toAnyMap(rec)) +// } catch { +// case e: Throwable => None +// } +// } +// Map[String, Any]((name -> arr)) +// } else { +// records.headOption match { +// case Some(head) => { +// try { +// JsonUtil.toAnyMap(head) +// } catch { +// case e: Throwable => emptyMap +// } +// } +// case _ => emptyMap +// } +// } } catch { case e: Throwable => { error(s"persist result ${name} error: ${e.getMessage}") - emptyMap +// emptyMap + Map[Long, Map[String, Any]]() } } } - case _ => emptyMap +// case _ => emptyMap + case _ => Map[Long, Map[String, Any]]() } } diff --git a/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/GriffinDslAdaptor.scala b/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/GriffinDslAdaptor.scala index 3148f97ad..c17871631 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/GriffinDslAdaptor.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/GriffinDslAdaptor.scala @@ -188,13 +188,16 @@ case class GriffinDslAdaptor(dataSourceNames: Seq[String], // 2. miss count val missTableName = "_miss_" val missColName = getNameOpt(details, AccuracyInfo._Miss).getOrElse(AccuracyInfo._Miss) - val missSql = processType match { - case BatchProcessType => { - s"SELECT COUNT(*) AS `${missColName}` FROM `${missRecordsName}`" - } - case StreamingProcessType => { - s"SELECT `${GroupByColumn.tmst}`, COUNT(*) AS `${missColName}` FROM `${missRecordsName}` GROUP BY `${GroupByColumn.tmst}`" - } +// val missSql = processType match { +// case BatchProcessType => { +// s"SELECT COUNT(*) AS `${missColName}` FROM `${missRecordsName}`" +// } +// case StreamingProcessType => { +// s"SELECT `${GroupByColumn.tmst}`, COUNT(*) AS `${missColName}` FROM `${missRecordsName}` GROUP BY `${GroupByColumn.tmst}`" +// } +// } + val missSql = { + s"SELECT `${GroupByColumn.tmst}`, COUNT(*) AS `${missColName}` FROM `${missRecordsName}` GROUP BY `${GroupByColumn.tmst}`" } val missStep = SparkSqlStep( missTableName, @@ -206,13 +209,16 @@ case class GriffinDslAdaptor(dataSourceNames: Seq[String], // 3. total count val totalTableName = "_total_" val totalColName = getNameOpt(details, AccuracyInfo._Total).getOrElse(AccuracyInfo._Total) - val totalSql = processType match { - case BatchProcessType => { - s"SELECT COUNT(*) AS `${totalColName}` FROM `${sourceName}`" - } - case StreamingProcessType => { - s"SELECT `${GroupByColumn.tmst}`, COUNT(*) AS `${totalColName}` FROM `${sourceName}` GROUP BY `${GroupByColumn.tmst}`" - } +// val totalSql = processType match { +// case BatchProcessType => { +// s"SELECT COUNT(*) AS `${totalColName}` FROM `${sourceName}`" +// } +// case StreamingProcessType => { +// s"SELECT `${GroupByColumn.tmst}`, COUNT(*) AS `${totalColName}` FROM `${sourceName}` GROUP BY `${GroupByColumn.tmst}`" +// } +// } + val totalSql = { + s"SELECT `${GroupByColumn.tmst}`, COUNT(*) AS `${totalColName}` FROM `${sourceName}` GROUP BY `${GroupByColumn.tmst}`" } val totalStep = SparkSqlStep( totalTableName, @@ -223,35 +229,71 @@ case class GriffinDslAdaptor(dataSourceNames: Seq[String], // 4. accuracy metric val matchedColName = getNameOpt(details, AccuracyInfo._Matched).getOrElse(AccuracyInfo._Matched) - val accuracyMetricSql = processType match { - case BatchProcessType => { - s""" - |SELECT `${totalTableName}`.`${totalColName}` AS `${totalColName}`, - |`${missTableName}`.`${missColName}` AS `${missColName}`, - |(`${totalColName}` - `${missColName}`) AS `${matchedColName}` - |FROM `${totalTableName}` JOIN `${missTableName}` - """.stripMargin - } - case StreamingProcessType => { - s""" - |SELECT `${missTableName}`.`${GroupByColumn.tmst}`, - |`${totalTableName}`.`${totalColName}` AS `${totalColName}`, - |`${missTableName}`.`${missColName}` AS `${missColName}`, - |(`${totalColName}` - `${missColName}`) AS `${matchedColName}` - |FROM `${totalTableName}` JOIN `${missTableName}` - |ON `${totalTableName}`.`${GroupByColumn.tmst}` = `${missTableName}`.`${GroupByColumn.tmst}` - """.stripMargin - } +// val accuracyMetricSql = processType match { +// case BatchProcessType => { +// s""" +// |SELECT `${totalTableName}`.`${totalColName}` AS `${totalColName}`, +// |`${missTableName}`.`${missColName}` AS `${missColName}`, +// |(`${totalColName}` - `${missColName}`) AS `${matchedColName}` +// |FROM `${totalTableName}` JOIN `${missTableName}` +// """.stripMargin +// } +// case StreamingProcessType => { +// s""" +// |SELECT `${missTableName}`.`${GroupByColumn.tmst}`, +// |`${totalTableName}`.`${totalColName}` AS `${totalColName}`, +// |`${missTableName}`.`${missColName}` AS `${missColName}`, +// |(`${totalColName}` - `${missColName}`) AS `${matchedColName}` +// |FROM `${totalTableName}` JOIN `${missTableName}` +// |ON `${totalTableName}`.`${GroupByColumn.tmst}` = `${missTableName}`.`${GroupByColumn.tmst}` +// """.stripMargin +// } +// } + + +// val accuracyMetricSql = { +// s""" +// |SELECT `${missTableName}`.`${GroupByColumn.tmst}`, +// |`${totalTableName}`.`${totalColName}` AS `${totalColName}`, +// |`${missTableName}`.`${missColName}` AS `${missColName}`, +// |(`${totalColName}` - `${missColName}`) AS `${matchedColName}` +// |FROM `${totalTableName}` JOIN `${missTableName}` +// |ON `${totalTableName}`.`${GroupByColumn.tmst}` = `${missTableName}`.`${GroupByColumn.tmst}` +// """.stripMargin +// } + val accuracyMetricSql = { + s""" + |SELECT `${missTableName}`.`${GroupByColumn.tmst}` AS `${GroupByColumn.tmst}`, + |`${totalTableName}`.`${totalColName}` AS `${totalColName}`, + |`${missTableName}`.`${missColName}` AS `${missColName}` + |FROM `${totalTableName}` FULL JOIN `${missTableName}` + |ON `${totalTableName}`.`${GroupByColumn.tmst}` = `${missTableName}`.`${GroupByColumn.tmst}` + """.stripMargin } val accuracyMetricName = resultName(details, AccuracyInfo._Accuracy) val accuracyMetricStep = SparkSqlStep( accuracyMetricName, accuracyMetricSql, details, +// resultPersistType(details, AccuracyInfo._Accuracy, MetricPersistType) + NonePersistType + ) + + // 5. test + val accuracyStep = DfOprStep( + accuracyMetricName, + "accuracy", + Map[String, Any]( + ("df.name" -> accuracyMetricName), + ("miss" -> missColName), + ("total" -> totalColName), + ("matched" -> matchedColName), + ("tmst" -> GroupByColumn.tmst) + ), resultPersistType(details, AccuracyInfo._Accuracy, MetricPersistType) ) - missRecordsStep :: missStep :: totalStep :: accuracyMetricStep :: Nil + missRecordsStep :: missStep :: totalStep :: accuracyMetricStep :: accuracyStep :: Nil } case ProfilingType => { val sourceName = getNameOpt(details, ProfilingInfo._Source) match { @@ -260,26 +302,35 @@ case class GriffinDslAdaptor(dataSourceNames: Seq[String], } val analyzer = ProfilingAnalyzer(expr.asInstanceOf[CombinedClause], sourceName) - // 1. select statement - val profilingSql = { - val selClause = analyzer.selectionExprs.map { sel => - val alias = sel match { - case s: AliasableExpr if (s.alias.nonEmpty) => s" AS ${s.alias.get}" - case _ => "" - } - s"${sel.desc}${alias}" - }.mkString(", ") + val selClause = analyzer.selectionExprs.map { sel => + val alias = sel match { + case s: AliasableExpr if (s.alias.nonEmpty) => s" AS ${s.alias.get}" + case _ => "" + } + s"${sel.desc}${alias}" + }.mkString(", ") - val tailClause = analyzer.tailsExprs.map(_.desc).mkString(" ") + val tailClause = analyzer.tailsExprs.map(_.desc).mkString(" ") - s"SELECT ${selClause} FROM ${sourceName} ${tailClause}" + // 1. select statement +// val profilingSql = processType match { +// case BatchProcessType => { +// s"SELECT ${selClause} FROM ${sourceName} ${tailClause}" +// } +// case StreamingProcessType => { +// s"SELECT ${GroupByColumn.tmst}, ${selClause} FROM ${sourceName} ${tailClause}" + +// s" GROUP BY ${GroupByColumn.tmst}" +// } +// } + val profilingSql = { + s"SELECT `${GroupByColumn.tmst}`, ${selClause} FROM ${sourceName} ${tailClause} GROUP BY `${GroupByColumn.tmst}`" } val profilingMetricName = resultName(details, ProfilingInfo._Profiling) val profilingStep = SparkSqlStep( profilingMetricName, profilingSql, details, - resultPersistType(details, ProfilingInfo._Profiling, RecordPersistType) + resultPersistType(details, ProfilingInfo._Profiling, MetricPersistType) ) profilingStep :: Nil diff --git a/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/SparkSqlAdaptor.scala b/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/SparkSqlAdaptor.scala index f4ec18720..f8032c461 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/SparkSqlAdaptor.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/SparkSqlAdaptor.scala @@ -18,8 +18,8 @@ under the License. */ package org.apache.griffin.measure.rules.adaptor -import org.apache.griffin.measure.algo.ProcessType -import org.apache.griffin.measure.rules.dsl.PersistType +import org.apache.griffin.measure.algo._ +import org.apache.griffin.measure.data.connector.GroupByColumn import org.apache.griffin.measure.rules.step._ case class SparkSqlAdaptor(processType: ProcessType) extends RuleAdaptor { @@ -29,7 +29,17 @@ case class SparkSqlAdaptor(processType: ProcessType) extends RuleAdaptor { } def adaptConcreteRuleStep(ruleStep: RuleStep): Seq[ConcreteRuleStep] = { ruleStep match { - case rs @ SparkSqlStep(_, _, _, _) => rs :: Nil + case rs @ SparkSqlStep(name, rule, details, persistType) => { + processType match { + case BatchProcessType => rs :: Nil + case StreamingProcessType => { + val repSel = rule.replaceFirst("(?i)select", s"SELECT `${GroupByColumn.tmst}`,") + val groupbyRule = repSel.concat(s" GROUP BY `${GroupByColumn.tmst}`") + val nrs = SparkSqlStep(name, groupbyRule, details, persistType) + nrs :: Nil + } + } + } case _ => Nil } } diff --git a/measure/src/main/scala/org/apache/griffin/measure/rules/preproc/PreProcRuleGenerator.scala b/measure/src/main/scala/org/apache/griffin/measure/rules/preproc/PreProcRuleGenerator.scala index d63567cad..571841c47 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rules/preproc/PreProcRuleGenerator.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rules/preproc/PreProcRuleGenerator.scala @@ -23,16 +23,20 @@ object PreProcRuleGenerator { val _name = "name" def genPreProcRules(rules: Seq[Map[String, Any]], suffix: String): Seq[Map[String, Any]] = { - rules.map { rule => - genPreProcRule(rule, suffix) + if (rules == null) Nil else { + rules.map { rule => + genPreProcRule(rule, suffix) + } } } def getRuleNames(rules: Seq[Map[String, Any]]): Seq[String] = { - rules.flatMap { rule => - rule.get(_name) match { - case Some(s: String) => Some(s) - case _ => None + if (rules == null) Nil else { + rules.flatMap { rule => + rule.get(_name) match { + case Some(s: String) => Some(s) + case _ => None + } } } } diff --git a/measure/src/main/scala/org/apache/griffin/measure/rules/step/ConcreteRuleStep.scala b/measure/src/main/scala/org/apache/griffin/measure/rules/step/ConcreteRuleStep.scala index d01f0290f..17fdb4479 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rules/step/ConcreteRuleStep.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rules/step/ConcreteRuleStep.scala @@ -24,12 +24,12 @@ trait ConcreteRuleStep extends RuleStep { val persistType: PersistType - def isGroupMetric: Boolean = { - val _GroupMetric = "group.metric" - details.get(_GroupMetric) match { - case Some(b: Boolean) => b - case _ => false - } - } +// def isGroupMetric: Boolean = { +// val _GroupMetric = "group.metric" +// details.get(_GroupMetric) match { +// case Some(b: Boolean) => b +// case _ => false +// } +// } } diff --git a/measure/src/test/resources/config-test-accuracy-streaming.json b/measure/src/test/resources/config-test-accuracy-streaming.json index 5a2ec3c38..95c3ee12b 100644 --- a/measure/src/test/resources/config-test-accuracy-streaming.json +++ b/measure/src/test/resources/config-test-accuracy-streaming.json @@ -110,8 +110,7 @@ }, "miss": "miss_count", "total": "total_count", - "matched": "matched_count", - "group.metric": true + "matched": "matched_count" } } ] diff --git a/measure/src/test/resources/config-test-profiling.json b/measure/src/test/resources/config-test-profiling.json index bd8644e3b..721dc4752 100644 --- a/measure/src/test/resources/config-test-profiling.json +++ b/measure/src/test/resources/config-test-profiling.json @@ -20,19 +20,6 @@ "evaluateRule": { "rules": [ - { - "dsl.type": "griffin-dsl", - "dq.type": "profiling", - "rule": "count(source.user_id), source.first_name group by source.first_name order by count(source.user_id) desc, source.first_name asc limit 3", - "details": { - "source": "source", - "profiling": { - "name": "prof", - "persist.type": "metric" - }, - "group.metric": true - } - }, { "dsl.type": "griffin-dsl", "dq.type": "profiling", From 1332f51e209db015dfb20cc8cbac588b6e89054f Mon Sep 17 00:00:00 2001 From: Lionel Liu Date: Wed, 13 Sep 2017 21:25:27 +0800 Subject: [PATCH 068/111] run --- .../measure/cache/metric/CacheMetric.scala | 27 ---- .../cache/metric/CacheMetricProcesser.scala | 72 ----------- .../cache/result/CacheResultProcesser.scala | 2 +- .../griffin/measure/metric/Metric.scala | 23 ---- .../measure/metric/MetricCompare.scala | 29 ----- .../measure/process/StreamingDqThread.scala | 24 ++-- .../process/engine/DataFrameOprEngine.scala | 47 +++++-- .../measure/process/engine/DqEngine.scala | 5 +- .../measure/process/engine/DqEngines.scala | 122 ++++++++++++------ .../process/engine/SparkSqlEngine.scala | 40 +++++- .../rules/adaptor/GriffinDslAdaptor.scala | 8 +- .../measure/process/BatchProcessTest.scala | 4 +- 12 files changed, 177 insertions(+), 226 deletions(-) delete mode 100644 measure/src/main/scala/org/apache/griffin/measure/cache/metric/CacheMetric.scala delete mode 100644 measure/src/main/scala/org/apache/griffin/measure/cache/metric/CacheMetricProcesser.scala delete mode 100644 measure/src/main/scala/org/apache/griffin/measure/metric/Metric.scala delete mode 100644 measure/src/main/scala/org/apache/griffin/measure/metric/MetricCompare.scala diff --git a/measure/src/main/scala/org/apache/griffin/measure/cache/metric/CacheMetric.scala b/measure/src/main/scala/org/apache/griffin/measure/cache/metric/CacheMetric.scala deleted file mode 100644 index b4a63a309..000000000 --- a/measure/src/main/scala/org/apache/griffin/measure/cache/metric/CacheMetric.scala +++ /dev/null @@ -1,27 +0,0 @@ -/* -Licensed to the Apache Software Foundation (ASF) under one -or more contributor license agreements. See the NOTICE file -distributed with this work for additional information -regarding copyright ownership. The ASF licenses this file -to you under the Apache License, Version 2.0 (the -"License"); you may not use this file except in compliance -with the License. You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, -software distributed under the License is distributed on an -"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -KIND, either express or implied. See the License for the -specific language governing permissions and limitations -under the License. -*/ -package org.apache.griffin.measure.cache.metric - -case class CacheMetric(timeGroup: Long, updateTime: Long, metric: Map[String, Any]) { - - def olderThan(ut: Long): Boolean = { - updateTime < ut - } - -} diff --git a/measure/src/main/scala/org/apache/griffin/measure/cache/metric/CacheMetricProcesser.scala b/measure/src/main/scala/org/apache/griffin/measure/cache/metric/CacheMetricProcesser.scala deleted file mode 100644 index 7da51dbf7..000000000 --- a/measure/src/main/scala/org/apache/griffin/measure/cache/metric/CacheMetricProcesser.scala +++ /dev/null @@ -1,72 +0,0 @@ -///* -//Licensed to the Apache Software Foundation (ASF) under one -//or more contributor license agreements. See the NOTICE file -//distributed with this work for additional information -//regarding copyright ownership. The ASF licenses this file -//to you under the Apache License, Version 2.0 (the -//"License"); you may not use this file except in compliance -//with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -//Unless required by applicable law or agreed to in writing, -//software distributed under the License is distributed on an -//"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -//KIND, either express or implied. See the License for the -//specific language governing permissions and limitations -//under the License. -//*/ -//package org.apache.griffin.measure.cache.metric -// -//import org.apache.griffin.measure.log.Loggable -//import org.apache.griffin.measure.result._ -// -//import scala.collection.mutable.{Map => MutableMap} -// -//case class CacheMetricProcesser() extends Loggable { -// -// val cacheGroup: MutableMap[Long, CacheMetric] = MutableMap() -// -// def genUpdateCacheMetric(timeGroup: Long, updateTime: Long, metric: Map[String, Any]): Option[CacheMetric] = { -// cacheGroup.get(timeGroup) match { -// case Some(cr) => { -// if (cr.olderThan(updateTime)) { -// val existMetric = cr.metric -// val newMetric = existMetric -// val newMetric = existMetric.update(result.asInstanceOf[existResult.T]) -// if (existResult.differsFrom(newResult)) { -// Some(CacheMetric(timeGroup, updateTime, newResult)) -// } else None -// } else None -// } -// case _ => { -// Some(CacheMetric(timeGroup, updateTime, result)) -// } -// } -// } -// -// def update(cr: CacheMetric): Unit = { -// val t = cr.timeGroup -// cacheGroup.get(t) match { -// case Some(c) => { -// if (c.olderThan(cr.updateTime)) cacheGroup += (t -> cr) -// } -// case _ => cacheGroup += (t -> cr) -// } -// } -// -// def getCacheResult(timeGroup: Long): Option[CacheMetric] = { -// cacheGroup.get(timeGroup) -// } -// -// def refresh(overtime: Long): Unit = { -// val curCacheGroup = cacheGroup.toMap -// val deadCache = curCacheGroup.filter { pr => -// val (_, cr) = pr -// cr.timeGroup < overtime || cr.result.eventual() -// } -// info(s"=== dead cache group count: ${deadCache.size} ===") -// deadCache.keySet.foreach(cacheGroup -= _) -// } -// -//} diff --git a/measure/src/main/scala/org/apache/griffin/measure/cache/result/CacheResultProcesser.scala b/measure/src/main/scala/org/apache/griffin/measure/cache/result/CacheResultProcesser.scala index 50d3adab6..9916e925d 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/cache/result/CacheResultProcesser.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/cache/result/CacheResultProcesser.scala @@ -23,7 +23,7 @@ import org.apache.griffin.measure.result._ import scala.collection.mutable.{Map => MutableMap} -case class CacheResultProcesser() extends Loggable { +object CacheResultProcesser extends Loggable { val cacheGroup: MutableMap[Long, CacheResult] = MutableMap() diff --git a/measure/src/main/scala/org/apache/griffin/measure/metric/Metric.scala b/measure/src/main/scala/org/apache/griffin/measure/metric/Metric.scala deleted file mode 100644 index d52991301..000000000 --- a/measure/src/main/scala/org/apache/griffin/measure/metric/Metric.scala +++ /dev/null @@ -1,23 +0,0 @@ -/* -Licensed to the Apache Software Foundation (ASF) under one -or more contributor license agreements. See the NOTICE file -distributed with this work for additional information -regarding copyright ownership. The ASF licenses this file -to you under the Apache License, Version 2.0 (the -"License"); you may not use this file except in compliance -with the License. You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, -software distributed under the License is distributed on an -"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -KIND, either express or implied. See the License for the -specific language governing permissions and limitations -under the License. -*/ -package org.apache.griffin.measure.metric - -case class Metric(value: Map[String, Any]) extends Serializable { - -} diff --git a/measure/src/main/scala/org/apache/griffin/measure/metric/MetricCompare.scala b/measure/src/main/scala/org/apache/griffin/measure/metric/MetricCompare.scala deleted file mode 100644 index 1e100eff5..000000000 --- a/measure/src/main/scala/org/apache/griffin/measure/metric/MetricCompare.scala +++ /dev/null @@ -1,29 +0,0 @@ -///* -//Licensed to the Apache Software Foundation (ASF) under one -//or more contributor license agreements. See the NOTICE file -//distributed with this work for additional information -//regarding copyright ownership. The ASF licenses this file -//to you under the Apache License, Version 2.0 (the -//"License"); you may not use this file except in compliance -//with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -//Unless required by applicable law or agreed to in writing, -//software distributed under the License is distributed on an -//"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -//KIND, either express or implied. See the License for the -//specific language governing permissions and limitations -//under the License. -//*/ -//package org.apache.griffin.measure.metric -// -//case class MetricCompare(config: Map[String, String]) extends Serializable { -// -// val _update -// -// def differ(opt1: Option[Metric], opt2: Option[Metric]): Boolean = { -// -// } -// -//} diff --git a/measure/src/main/scala/org/apache/griffin/measure/process/StreamingDqThread.scala b/measure/src/main/scala/org/apache/griffin/measure/process/StreamingDqThread.scala index 97bc619b3..9ce6a0ece 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/process/StreamingDqThread.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/process/StreamingDqThread.scala @@ -23,6 +23,7 @@ import java.util.concurrent.TimeUnit import org.apache.griffin.measure.algo._ import org.apache.griffin.measure.cache.info.{InfoCacheInstance, TimeInfoCache} +import org.apache.griffin.measure.cache.result.CacheResultProcesser import org.apache.griffin.measure.config.params.user.EvaluateRuleParam import org.apache.griffin.measure.data.source.DataSource import org.apache.griffin.measure.log.Loggable @@ -69,6 +70,9 @@ case class StreamingDqThread(dqEngines: DqEngines, TimeInfoCache.endTimeInfoCache + // clean old data + cleanData + val et = new Date().getTime appPersist.log(et, s"persist using time: ${et - ct} ms") @@ -85,18 +89,18 @@ case class StreamingDqThread(dqEngines: DqEngines, } // clean old data and old result cache -// def cleanData(): Unit = { -// try { + private def cleanData(): Unit = { + try { // sourceDataConnector.cleanOldData // targetDataConnector.cleanOldData -// -// val cleanTime = TimeInfoCache.getCleanTime -// cacheResultProcesser.refresh(cleanTime) -// } catch { -// case e: Throwable => error(s"clean data error: ${e.getMessage}") -// } -// } -// + + val cleanTime = TimeInfoCache.getCleanTime + CacheResultProcesser.refresh(cleanTime) + } catch { + case e: Throwable => error(s"clean data error: ${e.getMessage}") + } + } + // // calculate accuracy between source data and target data // private def accuracy(sourceData: RDD[(Product, (Map[String, Any], Map[String, Any]))], // targetData: RDD[(Product, (Map[String, Any], Map[String, Any]))], diff --git a/measure/src/main/scala/org/apache/griffin/measure/process/engine/DataFrameOprEngine.scala b/measure/src/main/scala/org/apache/griffin/measure/process/engine/DataFrameOprEngine.scala index 81578c5f8..659a671ff 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/process/engine/DataFrameOprEngine.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/process/engine/DataFrameOprEngine.scala @@ -70,7 +70,7 @@ case class DataFrameOprEngine(sqlContext: SQLContext, @transient ssc: StreamingC } } - def persistRecords(ruleStep: ConcreteRuleStep, persistFactory: PersistFactory): Boolean = { +// def persistRecords(ruleStep: ConcreteRuleStep, timeGroups: Iterable[Long], persistFactory: PersistFactory): Boolean = { // val curTime = new Date().getTime // ruleStep match { // case DfOprStep(name, _, _, RecordPersistType) => { @@ -93,7 +93,31 @@ case class DataFrameOprEngine(sqlContext: SQLContext, @transient ssc: StreamingC // } // case _ => false // } - true +// true +// } + + def collectRecords(ruleStep: ConcreteRuleStep, timeGroups: Iterable[Long]): Map[Long, RDD[String]] = { + ruleStep match { + case DfOprStep(name, _, _, RecordPersistType) => { + try { + val pdf = sqlContext.table(s"`${name}`") + timeGroups.flatMap { timeGroup => + try { + val rdd = pdf.filter(s"`${GroupByColumn.tmst}` = ${timeGroup}").toJSON + Some((timeGroup, rdd)) + } catch { + case e: Throwable => None + } + }.toMap + } catch { + case e: Throwable => { + error(s"persist result ${name} error: ${e.getMessage}") + Map[Long, RDD[String]]() + } + } + } + case _ => Map[Long, RDD[String]]() + } } def collectMetrics(ruleStep: ConcreteRuleStep): Map[Long, Map[String, Any]] = { @@ -208,8 +232,6 @@ object DataFrameOprs { sqlContext.read.json(rdd) } - final val cacheResultProcesser = CacheResultProcesser() - def accuracy(sqlContext: SQLContext, details: Map[String, Any]): DataFrame = { val _dfName = "df.name" val _miss = "miss" @@ -233,23 +255,26 @@ object DataFrameOprs { } val df = sqlContext.table(s"`${dfName}`") - val results = df.map { row => + df.show(10) + val results = df.flatMap { row => val t = getLong(row, tmst) - val missCount = getLong(row, miss) - val totalCount = getLong(row, total) - val ar = AccuracyResult(missCount, totalCount) - (t, ar) + if (t > 0) { + val missCount = getLong(row, miss) + val totalCount = getLong(row, total) + val ar = AccuracyResult(missCount, totalCount) + Some((t, ar)) + } else None }.collect val updateResults = results.flatMap { pair => val (t, result) = pair - val updatedCacheResultOpt = cacheResultProcesser.genUpdateCacheResult(t, updateTime, result) + val updatedCacheResultOpt = CacheResultProcesser.genUpdateCacheResult(t, updateTime, result) updatedCacheResultOpt } // update updateResults.foreach { r => - cacheResultProcesser.update(r) + CacheResultProcesser.update(r) } val schema = StructType(Array( diff --git a/measure/src/main/scala/org/apache/griffin/measure/process/engine/DqEngine.scala b/measure/src/main/scala/org/apache/griffin/measure/process/engine/DqEngine.scala index c90240704..24a97d682 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/process/engine/DqEngine.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/process/engine/DqEngine.scala @@ -24,6 +24,7 @@ import org.apache.griffin.measure.log.Loggable import org.apache.griffin.measure.persist.{Persist, PersistFactory} import org.apache.griffin.measure.rules.dsl._ import org.apache.griffin.measure.rules.step._ +import org.apache.spark.rdd.RDD import org.apache.spark.sql.DataFrame trait DqEngine extends Loggable with Serializable { @@ -34,7 +35,9 @@ trait DqEngine extends Loggable with Serializable { // def persistResults(ruleSteps: Seq[ConcreteRuleStep], persist: Persist, persistType: PersistType): Boolean - def persistRecords(ruleStep: ConcreteRuleStep, persistFactory: PersistFactory): Boolean +// def persistRecords(ruleStep: ConcreteRuleStep, timeGroups: Iterable[Long], persistFactory: PersistFactory): Boolean + def collectMetrics(ruleStep: ConcreteRuleStep): Map[Long, Map[String, Any]] + def collectRecords(ruleStep: ConcreteRuleStep, timeGroups: Iterable[Long]): Map[Long, RDD[String]] } diff --git a/measure/src/main/scala/org/apache/griffin/measure/process/engine/DqEngines.scala b/measure/src/main/scala/org/apache/griffin/measure/process/engine/DqEngines.scala index 5da323724..58748e175 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/process/engine/DqEngines.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/process/engine/DqEngines.scala @@ -25,6 +25,7 @@ import org.apache.griffin.measure.log.Loggable import org.apache.griffin.measure.persist.{Persist, PersistFactory} import org.apache.griffin.measure.rules.dsl._ import org.apache.griffin.measure.rules.step._ +import org.apache.spark.rdd.RDD case class DqEngines(engines: Seq[DqEngine]) extends DqEngine { @@ -46,46 +47,80 @@ case class DqEngines(engines: Seq[DqEngine]) extends DqEngine { } def persistAllResults(ruleSteps: Seq[ConcreteRuleStep], persistFactory: PersistFactory): Unit = { - // 1. group by same persist types - val groupedRuleSteps = ruleSteps.groupBy(_.persistType) - - // 2. persist results in order [metric, record] - persistOrder.foreach { prstType => - val steps = groupedRuleSteps.get(prstType) match { - case Some(a) => a - case _ => Nil - } - prstType match { - case MetricPersistType => { -// val metrics = steps.foldLeft(Map[String, Any]())(_ ++ collectMetrics(_)) - val metrics: Map[Long, Map[String, Any]] = { - steps.foldLeft(Map[Long, Map[String, Any]]()) { (ret, step) => - val metrics = collectMetrics(step) - metrics.foldLeft(ret) { (total, pair) => - val (k, v) = pair - ret.get(k) match { - case Some(map) => ret + (k -> (map ++ v)) - case _ => ret + pair - } - } - } - } - metrics.foreach { pair => - val (t, metric) = pair - val persist = persistFactory.getPersists(t) - persist.persistMetrics(metric) - } - } - case RecordPersistType => { - steps.foreach { ruleStep => - persistRecords(ruleStep, persistFactory) + // 1. persist metric + val metricSteps = ruleSteps.filter(_.persistType == MetricPersistType) + val allMetrics: Map[Long, Map[String, Any]] = { + metricSteps.foldLeft(Map[Long, Map[String, Any]]()) { (ret, step) => + val metrics = collectMetrics(step) + metrics.foldLeft(ret) { (total, pair) => + val (k, v) = pair + total.get(k) match { + case Some(map) => total + (k -> (map ++ v)) + case _ => total + pair } } - case _ => { - warn(s"${prstType} is not persistable") - } } } + println(allMetrics) + val updateTimeGroups = allMetrics.keys + allMetrics.foreach { pair => + val (t, metric) = pair + val persist = persistFactory.getPersists(t) + persist.persistMetrics(metric) + } + + // 2. persist record + val recordSteps = ruleSteps.filter(_.persistType == RecordPersistType) + recordSteps.foreach { step => + val name = step.name + val records = collectRecords(step, updateTimeGroups) + records.foreach { pair => + val (t, recs) = pair + val persist = persistFactory.getPersists(t) + persist.persistRecords(recs, name) + } + } + + // 1. group by same persist types +// val groupedRuleSteps = ruleSteps.groupBy(_.persistType) +// +// // 2. persist results in order [metric, record] +// persistOrder.foreach { prstType => +// val steps = groupedRuleSteps.get(prstType) match { +// case Some(a) => a +// case _ => Nil +// } +// prstType match { +// case MetricPersistType => { +//// val metrics = steps.foldLeft(Map[String, Any]())(_ ++ collectMetrics(_)) +// val allMetrics: Map[Long, Map[String, Any]] = { +// steps.foldLeft(Map[Long, Map[String, Any]]()) { (ret, step) => +// val metrics = collectMetrics(step) +// metrics.foldLeft(ret) { (total, pair) => +// val (k, v) = pair +// total.get(k) match { +// case Some(map) => total + (k -> (map ++ v)) +// case _ => total + pair +// } +// } +// } +// } +// allMetrics.foreach { pair => +// val (t, metric) = pair +// val persist = persistFactory.getPersists(t) +// persist.persistMetrics(metric) +// } +// } +// case RecordPersistType => { +// steps.foreach { ruleStep => +// persistRecords(ruleStep, persistFactory) +// } +// } +// case _ => { +// warn(s"${prstType} is not persistable") +// } +// } +// } } // def genDataSource(dataSourceParam: DataSourceParam): Option[DirectDataSource] = { @@ -104,11 +139,18 @@ case class DqEngines(engines: Seq[DqEngine]) extends DqEngine { ret } - def persistRecords(ruleStep: ConcreteRuleStep, persistFactory: PersistFactory): Boolean = { - val ret = engines.foldLeft(false) { (done, engine) => - done || engine.persistRecords(ruleStep, persistFactory) +// def persistRecords(ruleStep: ConcreteRuleStep, timeGroups: Iterable[Long], persistFactory: PersistFactory): Boolean = { +// val ret = engines.foldLeft(false) { (done, engine) => +// done || engine.persistRecords(ruleStep, timeGroups, persistFactory) +// } +// if (!ret) error(s"persist result warn: no dq engine support ${ruleStep}") +// ret +// } + def collectRecords(ruleStep: ConcreteRuleStep, timeGroups: Iterable[Long]): Map[Long, RDD[String]] = { + val ret = engines.foldLeft(Map[Long, RDD[String]]()) { (ret, engine) => + ret ++ engine.collectRecords(ruleStep, timeGroups) } - if (!ret) error(s"persist result warn: no dq engine support ${ruleStep}") + if (ret.isEmpty) warn(s"collect records warn: no records collected for ${ruleStep}") ret } def collectMetrics(ruleStep: ConcreteRuleStep): Map[Long, Map[String, Any]] = { diff --git a/measure/src/main/scala/org/apache/griffin/measure/process/engine/SparkSqlEngine.scala b/measure/src/main/scala/org/apache/griffin/measure/process/engine/SparkSqlEngine.scala index ec77fe268..e24909a3d 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/process/engine/SparkSqlEngine.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/process/engine/SparkSqlEngine.scala @@ -27,7 +27,8 @@ import org.apache.griffin.measure.persist.{Persist, PersistFactory} import org.apache.griffin.measure.rules.dsl._ import org.apache.griffin.measure.rules.step._ import org.apache.griffin.measure.utils.JsonUtil -import org.apache.spark.sql.{DataFrame, SQLContext} +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.{DataFrame, GroupedData, SQLContext} import org.apache.spark.streaming.StreamingContext case class SparkSqlEngine(sqlContext: SQLContext, @transient ssc: StreamingContext @@ -55,7 +56,7 @@ case class SparkSqlEngine(sqlContext: SQLContext, @transient ssc: StreamingConte } } - def persistRecords(ruleStep: ConcreteRuleStep, persistFactory: PersistFactory): Boolean = { +// def persistRecords(ruleStep: ConcreteRuleStep, timeGroups: Iterable[Long], persistFactory: PersistFactory): Boolean = { // val curTime = new Date().getTime // ruleStep match { // case SparkSqlStep(name, _, _, RecordPersistType) => { @@ -63,10 +64,14 @@ case class SparkSqlEngine(sqlContext: SQLContext, @transient ssc: StreamingConte // val pdf = sqlContext.table(s"`${name}`") // val records = pdf.toJSON // -// persist.persistRecords(records, name) +// timeGroups.foreach { timeGroup => +// val persist = persistFactory.getPersists(timeGroup) // -// val recordLog = s"[ ${name} ] persist records" -// persist.log(curTime, recordLog) +// persist.persistRecords(records, name) +// +//// val recordLog = s"[ ${name} ] persist records" +//// persist.log(curTime, recordLog) +// } // // true // } catch { @@ -78,7 +83,30 @@ case class SparkSqlEngine(sqlContext: SQLContext, @transient ssc: StreamingConte // } // case _ => false // } - true +// } + + def collectRecords(ruleStep: ConcreteRuleStep, timeGroups: Iterable[Long]): Map[Long, RDD[String]] = { + ruleStep match { + case SparkSqlStep(name, _, _, RecordPersistType) => { + try { + val pdf = sqlContext.table(s"`${name}`") + timeGroups.flatMap { timeGroup => + try { + val rdd = pdf.filter(s"`${GroupByColumn.tmst}` = ${timeGroup}").toJSON + Some((timeGroup, rdd)) + } catch { + case e: Throwable => None + } + }.toMap + } catch { + case e: Throwable => { + error(s"persist result ${name} error: ${e.getMessage}") + Map[Long, RDD[String]]() + } + } + } + case _ => Map[Long, RDD[String]]() + } } def collectMetrics(ruleStep: ConcreteRuleStep): Map[Long, Map[String, Any]] = { diff --git a/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/GriffinDslAdaptor.scala b/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/GriffinDslAdaptor.scala index c17871631..2b7dd657f 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/GriffinDslAdaptor.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/GriffinDslAdaptor.scala @@ -263,9 +263,9 @@ case class GriffinDslAdaptor(dataSourceNames: Seq[String], // } val accuracyMetricSql = { s""" - |SELECT `${missTableName}`.`${GroupByColumn.tmst}` AS `${GroupByColumn.tmst}`, - |`${totalTableName}`.`${totalColName}` AS `${totalColName}`, - |`${missTableName}`.`${missColName}` AS `${missColName}` + |SELECT `${totalTableName}`.`${GroupByColumn.tmst}` AS `${GroupByColumn.tmst}`, + |`${missTableName}`.`${missColName}` AS `${missColName}`, + |`${totalTableName}`.`${totalColName}` AS `${totalColName}` |FROM `${totalTableName}` FULL JOIN `${missTableName}` |ON `${totalTableName}`.`${GroupByColumn.tmst}` = `${missTableName}`.`${GroupByColumn.tmst}` """.stripMargin @@ -279,7 +279,7 @@ case class GriffinDslAdaptor(dataSourceNames: Seq[String], NonePersistType ) - // 5. test + // 5. accuracy metric filter val accuracyStep = DfOprStep( accuracyMetricName, "accuracy", diff --git a/measure/src/test/scala/org/apache/griffin/measure/process/BatchProcessTest.scala b/measure/src/test/scala/org/apache/griffin/measure/process/BatchProcessTest.scala index 41e61c5ad..3a2b3e84a 100644 --- a/measure/src/test/scala/org/apache/griffin/measure/process/BatchProcessTest.scala +++ b/measure/src/test/scala/org/apache/griffin/measure/process/BatchProcessTest.scala @@ -36,8 +36,8 @@ import scala.util.{Failure, Success, Try} class BatchProcessTest extends FunSuite with Matchers with BeforeAndAfter with Loggable { val envFile = "src/test/resources/env-test.json" - val confFile = "src/test/resources/config-test-profiling.json" -// val confFile = "src/test/resources/config-test-accuracy.json" +// val confFile = "src/test/resources/config-test-profiling.json" + val confFile = "src/test/resources/config-test-accuracy.json" val envFsType = "local" val userFsType = "local" From 2220fa86dbf8eef01f9200276ac9be647b0c53f3 Mon Sep 17 00:00:00 2001 From: Lionel Liu Date: Wed, 13 Sep 2017 23:01:37 +0800 Subject: [PATCH 069/111] waiting for old data update --- .../data/connector/DataConnector.scala | 4 +- .../measure/data/source/DataSourceCache.scala | 31 ++++++ .../measure/persist/LoggerPersist.scala | 1 + .../measure/process/BatchDqProcess.scala | 7 +- .../measure/process/StreamingDqThread.scala | 10 +- .../process/engine/DataFrameOprEngine.scala | 7 +- .../measure/process/engine/DqEngines.scala | 98 +++++++++++-------- .../process/engine/SparkSqlEngine.scala | 6 +- .../measure/rules/adaptor/AdaptPhase.scala | 25 +++++ .../rules/adaptor/DataFrameOprAdaptor.scala | 7 +- .../rules/adaptor/GriffinDslAdaptor.scala | 36 ++++--- .../measure/rules/adaptor/RuleAdaptor.scala | 4 +- .../rules/adaptor/RuleAdaptorGroup.scala | 16 +-- .../rules/adaptor/SparkSqlAdaptor.scala | 17 ++-- .../measure/rules/step/ConcreteRuleStep.scala | 2 + .../measure/rules/step/DfOprStep.scala | 3 +- .../measure/rules/step/GriffinDslStep.scala | 3 +- .../measure/rules/step/SparkSqlStep.scala | 3 +- .../config-test-accuracy-streaming.json | 3 +- 19 files changed, 189 insertions(+), 94 deletions(-) create mode 100644 measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/AdaptPhase.scala diff --git a/measure/src/main/scala/org/apache/griffin/measure/data/connector/DataConnector.scala b/measure/src/main/scala/org/apache/griffin/measure/data/connector/DataConnector.scala index 6beaf18d0..9a6ccb2ca 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/data/connector/DataConnector.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/data/connector/DataConnector.scala @@ -24,7 +24,7 @@ import org.apache.griffin.measure.algo._ import org.apache.griffin.measure.config.params.user.DataConnectorParam import org.apache.griffin.measure.log.Loggable import org.apache.griffin.measure.process.engine._ -import org.apache.griffin.measure.rules.adaptor.RuleAdaptorGroup +import org.apache.griffin.measure.rules.adaptor.{PreProcPhase, RuleAdaptorGroup, RunPhase} import org.apache.griffin.measure.rules.dsl._ import org.apache.griffin.measure.rules.preproc.PreProcRuleGenerator import org.apache.spark.rdd.RDD @@ -64,7 +64,7 @@ trait DataConnector extends Loggable with Serializable { df.registerTempTable(thisTable) // generate rule steps - val ruleSteps = RuleAdaptorGroup.genConcreteRuleSteps(preProcRules, DslType("spark-sql"), BatchProcessType) + val ruleSteps = RuleAdaptorGroup.genConcreteRuleSteps(preProcRules, DslType("spark-sql"), PreProcPhase) // run rules dqEngines.runRuleSteps(ruleSteps) diff --git a/measure/src/main/scala/org/apache/griffin/measure/data/source/DataSourceCache.scala b/measure/src/main/scala/org/apache/griffin/measure/data/source/DataSourceCache.scala index c382dee7b..953b30789 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/data/source/DataSourceCache.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/data/source/DataSourceCache.scala @@ -138,6 +138,37 @@ case class DataSourceCache(sqlContext: SQLContext, param: Map[String, Any], } } + def updateOldData(t: Long, oldData: Iterable[Map[String, Any]]): Unit = { + // fixme + // parallel process different time groups, lock is unnecessary +// val ptns = getPartition(t) +// val ptnsPath = genPartitionHdfsPath(ptns) +// val dirPath = s"${filePath}/${ptnsPath}" +// val dataFileName = s"${t}" +// val dataFilePath = HdfsUtil.getHdfsFilePath(dirPath, dataFileName) +// +// try { +// // remove out time old data +// HdfsFileDumpUtil.remove(dirPath, dataFileName, true) +// +// // save updated old data +// if (oldData.size > 0) { +// val recordDatas = oldData.flatMap { dt => +// encode(dt, t) +// } +// val dumped = HdfsFileDumpUtil.dump(dataFilePath, recordDatas, rowSepLiteral) +// } +// } catch { +// case e: Throwable => error(s"update old data error: ${e.getMessage}") +// } + } + + override protected def genCleanTime(ms: Long): Long = { + val minPartitionUnit = partitionUnits.last + val t1 = TimeUtil.timeToUnit(ms, minPartitionUnit) + val t2 = TimeUtil.timeFromUnit(t1, minPartitionUnit) + t2 + } private def getPartition(ms: Long): List[Long] = { partitionUnits.map { unit => diff --git a/measure/src/main/scala/org/apache/griffin/measure/persist/LoggerPersist.scala b/measure/src/main/scala/org/apache/griffin/measure/persist/LoggerPersist.scala index 07774b720..b89d3a024 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/persist/LoggerPersist.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/persist/LoggerPersist.scala @@ -116,6 +116,7 @@ case class LoggerPersist(config: Map[String, Any], metricName: String, timeStamp } def persistRecords(records: RDD[String], name: String): Unit = { + println(s"${name} [${timeStamp}] records: ") try { val recordCount = records.count.toInt val count = if (maxLogLines < 0) recordCount else scala.math.min(maxLogLines, recordCount) diff --git a/measure/src/main/scala/org/apache/griffin/measure/process/BatchDqProcess.scala b/measure/src/main/scala/org/apache/griffin/measure/process/BatchDqProcess.scala index ba0764ff8..4b4678ba1 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/process/BatchDqProcess.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/process/BatchDqProcess.scala @@ -27,7 +27,7 @@ import org.apache.griffin.measure.config.params.user._ import org.apache.griffin.measure.data.source.DataSourceFactory import org.apache.griffin.measure.persist.{Persist, PersistFactory} import org.apache.griffin.measure.process.engine.{DqEngineFactory, SparkSqlEngine} -import org.apache.griffin.measure.rules.adaptor.RuleAdaptorGroup +import org.apache.griffin.measure.rules.adaptor.{RuleAdaptorGroup, RunPhase} import org.apache.griffin.measure.rules.udf.GriffinUdfs import org.apache.griffin.measure.utils.JsonUtil import org.apache.spark.sql.SQLContext @@ -87,13 +87,14 @@ case class BatchDqProcess(allParam: AllParam) extends DqProcess { dqEngines.loadData(dataSources, startTime) // generate rule steps - val ruleSteps = RuleAdaptorGroup.genConcreteRuleSteps(userParam.evaluateRuleParam, BatchProcessType) + val ruleSteps = RuleAdaptorGroup.genConcreteRuleSteps(userParam.evaluateRuleParam, RunPhase) // run rules dqEngines.runRuleSteps(ruleSteps) // persist results - dqEngines.persistAllResults(ruleSteps, persistFactory) + val timeGroups = dqEngines.persistAllMetrics(ruleSteps, persistFactory) + dqEngines.persistAllRecords(ruleSteps, persistFactory, timeGroups) // end time val endTime = new Date().getTime diff --git a/measure/src/main/scala/org/apache/griffin/measure/process/StreamingDqThread.scala b/measure/src/main/scala/org/apache/griffin/measure/process/StreamingDqThread.scala index 9ce6a0ece..1f19840a7 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/process/StreamingDqThread.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/process/StreamingDqThread.scala @@ -29,7 +29,7 @@ import org.apache.griffin.measure.data.source.DataSource import org.apache.griffin.measure.log.Loggable import org.apache.griffin.measure.persist.{Persist, PersistFactory} import org.apache.griffin.measure.process.engine.DqEngines -import org.apache.griffin.measure.rules.adaptor.RuleAdaptorGroup +import org.apache.griffin.measure.rules.adaptor.{RuleAdaptorGroup, RunPhase} case class StreamingDqThread(dqEngines: DqEngines, dataSources: Seq[DataSource], @@ -57,7 +57,7 @@ case class StreamingDqThread(dqEngines: DqEngines, dqEngines.loadData(dataSources, st) // generate rule steps - val ruleSteps = RuleAdaptorGroup.genConcreteRuleSteps(evaluateRuleParam, StreamingProcessType) + val ruleSteps = RuleAdaptorGroup.genConcreteRuleSteps(evaluateRuleParam, RunPhase) // run rules dqEngines.runRuleSteps(ruleSteps) @@ -65,8 +65,10 @@ case class StreamingDqThread(dqEngines: DqEngines, val ct = new Date().getTime appPersist.log(ct, s"calculation using time: ${ct - st} ms") - // persist results - dqEngines.persistAllResults(ruleSteps, persistFactory) + // persist results and cache records + val timeGroups = dqEngines.persistAllMetrics(ruleSteps, persistFactory) + dqEngines.updateDataSources(ruleSteps, dataSources, timeGroups) + dqEngines.persistAllRecords(ruleSteps, persistFactory, timeGroups) TimeInfoCache.endTimeInfoCache diff --git a/measure/src/main/scala/org/apache/griffin/measure/process/engine/DataFrameOprEngine.scala b/measure/src/main/scala/org/apache/griffin/measure/process/engine/DataFrameOprEngine.scala index 659a671ff..f49da08c7 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/process/engine/DataFrameOprEngine.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/process/engine/DataFrameOprEngine.scala @@ -43,7 +43,7 @@ case class DataFrameOprEngine(sqlContext: SQLContext, @transient ssc: StreamingC def runRuleStep(ruleStep: ConcreteRuleStep): Boolean = { ruleStep match { - case DfOprStep(name, rule, details, _) => { + case DfOprStep(name, rule, details, _, _) => { try { rule match { case DataFrameOprs._fromJson => { @@ -98,7 +98,7 @@ case class DataFrameOprEngine(sqlContext: SQLContext, @transient ssc: StreamingC def collectRecords(ruleStep: ConcreteRuleStep, timeGroups: Iterable[Long]): Map[Long, RDD[String]] = { ruleStep match { - case DfOprStep(name, _, _, RecordPersistType) => { + case DfOprStep(name, _, _, RecordPersistType, _) => { try { val pdf = sqlContext.table(s"`${name}`") timeGroups.flatMap { timeGroup => @@ -123,7 +123,7 @@ case class DataFrameOprEngine(sqlContext: SQLContext, @transient ssc: StreamingC def collectMetrics(ruleStep: ConcreteRuleStep): Map[Long, Map[String, Any]] = { val emptyMap = Map[String, Any]() ruleStep match { - case DfOprStep(name, _, _, MetricPersistType) => { + case DfOprStep(name, _, _, MetricPersistType, _) => { try { val pdf = sqlContext.table(s"`${name}`") val records = pdf.toJSON.collect() @@ -255,7 +255,6 @@ object DataFrameOprs { } val df = sqlContext.table(s"`${dfName}`") - df.show(10) val results = df.flatMap { row => val t = getLong(row, tmst) if (t > 0) { diff --git a/measure/src/main/scala/org/apache/griffin/measure/process/engine/DqEngines.scala b/measure/src/main/scala/org/apache/griffin/measure/process/engine/DqEngines.scala index 58748e175..b6a40c73f 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/process/engine/DqEngines.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/process/engine/DqEngines.scala @@ -46,8 +46,8 @@ case class DqEngines(engines: Seq[DqEngine]) extends DqEngine { } } - def persistAllResults(ruleSteps: Seq[ConcreteRuleStep], persistFactory: PersistFactory): Unit = { - // 1. persist metric + def persistAllMetrics(ruleSteps: Seq[ConcreteRuleStep], persistFactory: PersistFactory + ): Iterable[Long] = { val metricSteps = ruleSteps.filter(_.persistType == MetricPersistType) val allMetrics: Map[Long, Map[String, Any]] = { metricSteps.foldLeft(Map[Long, Map[String, Any]]()) { (ret, step) => @@ -61,67 +61,79 @@ case class DqEngines(engines: Seq[DqEngine]) extends DqEngine { } } } - println(allMetrics) val updateTimeGroups = allMetrics.keys allMetrics.foreach { pair => val (t, metric) = pair val persist = persistFactory.getPersists(t) persist.persistMetrics(metric) } + updateTimeGroups + } - // 2. persist record + def persistAllRecords(ruleSteps: Seq[ConcreteRuleStep], persistFactory: PersistFactory, + timeGroups: Iterable[Long]): Unit = { val recordSteps = ruleSteps.filter(_.persistType == RecordPersistType) recordSteps.foreach { step => val name = step.name - val records = collectRecords(step, updateTimeGroups) + val records = collectRecords(step, timeGroups) records.foreach { pair => val (t, recs) = pair val persist = persistFactory.getPersists(t) persist.persistRecords(recs, name) } } + } - // 1. group by same persist types -// val groupedRuleSteps = ruleSteps.groupBy(_.persistType) -// -// // 2. persist results in order [metric, record] -// persistOrder.foreach { prstType => -// val steps = groupedRuleSteps.get(prstType) match { -// case Some(a) => a -// case _ => Nil + def updateDataSources(ruleSteps: Seq[ConcreteRuleStep], dataSources: Seq[DataSource], + timeGroups: Iterable[Long]): Unit = { + // fixme +// val recordSteps = ruleSteps.filter(_.persistType == RecordPersistType) +// recordSteps.foreach { step => +// val name = step.name +// val records = collectRecords(step, timeGroups) +// records.foreach { pair => +// val (t, recs) = pair +// dataSources +// val persist = persistFactory.getPersists(t) +// persist.persistRecords(recs, name) // } -// prstType match { -// case MetricPersistType => { -//// val metrics = steps.foldLeft(Map[String, Any]())(_ ++ collectMetrics(_)) -// val allMetrics: Map[Long, Map[String, Any]] = { -// steps.foldLeft(Map[Long, Map[String, Any]]()) { (ret, step) => -// val metrics = collectMetrics(step) -// metrics.foldLeft(ret) { (total, pair) => -// val (k, v) = pair -// total.get(k) match { -// case Some(map) => total + (k -> (map ++ v)) -// case _ => total + pair -// } -// } -// } -// } -// allMetrics.foreach { pair => -// val (t, metric) = pair -// val persist = persistFactory.getPersists(t) -// persist.persistMetrics(metric) -// } -// } -// case RecordPersistType => { -// steps.foreach { ruleStep => -// persistRecords(ruleStep, persistFactory) +// } + } + +// def persistAllResults(ruleSteps: Seq[ConcreteRuleStep], persistFactory: PersistFactory): Unit = { +// // 1. persist metric +// val metricSteps = ruleSteps.filter(_.persistType == MetricPersistType) +// val allMetrics: Map[Long, Map[String, Any]] = { +// metricSteps.foldLeft(Map[Long, Map[String, Any]]()) { (ret, step) => +// val metrics = collectMetrics(step) +// metrics.foldLeft(ret) { (total, pair) => +// val (k, v) = pair +// total.get(k) match { +// case Some(map) => total + (k -> (map ++ v)) +// case _ => total + pair // } // } -// case _ => { -// warn(s"${prstType} is not persistable") -// } // } // } - } +// val updateTimeGroups = allMetrics.keys +// allMetrics.foreach { pair => +// val (t, metric) = pair +// val persist = persistFactory.getPersists(t) +// persist.persistMetrics(metric) +// } +// +// // 2. persist record +// val recordSteps = ruleSteps.filter(_.persistType == RecordPersistType) +// recordSteps.foreach { step => +// val name = step.name +// val records = collectRecords(step, updateTimeGroups) +// records.foreach { pair => +// val (t, recs) = pair +// val persist = persistFactory.getPersists(t) +// persist.persistRecords(recs, name) +// } +// } +// } // def genDataSource(dataSourceParam: DataSourceParam): Option[DirectDataSource] = { // val ret = engines.foldLeft(None: Option[DirectDataSource]) { (dsOpt, engine) => @@ -150,14 +162,14 @@ case class DqEngines(engines: Seq[DqEngine]) extends DqEngine { val ret = engines.foldLeft(Map[Long, RDD[String]]()) { (ret, engine) => ret ++ engine.collectRecords(ruleStep, timeGroups) } - if (ret.isEmpty) warn(s"collect records warn: no records collected for ${ruleStep}") +// if (ret.isEmpty) warn(s"collect records warn: no records collected for ${ruleStep}") ret } def collectMetrics(ruleStep: ConcreteRuleStep): Map[Long, Map[String, Any]] = { val ret = engines.foldLeft(Map[Long, Map[String, Any]]()) { (ret, engine) => ret ++ engine.collectMetrics(ruleStep) } - if (ret.isEmpty) warn(s"collect metrics warn: no metrics collected for ${ruleStep}") +// if (ret.isEmpty) warn(s"collect metrics warn: no metrics collected for ${ruleStep}") ret // val ret = engines.foldLeft(Map[Long, Map[String, Any]]()) { (ret, engine) => // val metrics: Map[Long, Map[String, Any]] = engine.collectMetrics(ruleStep) diff --git a/measure/src/main/scala/org/apache/griffin/measure/process/engine/SparkSqlEngine.scala b/measure/src/main/scala/org/apache/griffin/measure/process/engine/SparkSqlEngine.scala index e24909a3d..addc422d8 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/process/engine/SparkSqlEngine.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/process/engine/SparkSqlEngine.scala @@ -40,7 +40,7 @@ case class SparkSqlEngine(sqlContext: SQLContext, @transient ssc: StreamingConte def runRuleStep(ruleStep: ConcreteRuleStep): Boolean = { ruleStep match { - case SparkSqlStep(name, rule, _, _) => { + case SparkSqlStep(name, rule, _, _, _) => { try { val rdf = sqlContext.sql(rule) rdf.registerTempTable(name) @@ -87,7 +87,7 @@ case class SparkSqlEngine(sqlContext: SQLContext, @transient ssc: StreamingConte def collectRecords(ruleStep: ConcreteRuleStep, timeGroups: Iterable[Long]): Map[Long, RDD[String]] = { ruleStep match { - case SparkSqlStep(name, _, _, RecordPersistType) => { + case SparkSqlStep(name, _, _, RecordPersistType, _) => { try { val pdf = sqlContext.table(s"`${name}`") timeGroups.flatMap { timeGroup => @@ -112,7 +112,7 @@ case class SparkSqlEngine(sqlContext: SQLContext, @transient ssc: StreamingConte def collectMetrics(ruleStep: ConcreteRuleStep): Map[Long, Map[String, Any]] = { val emptyMap = Map[String, Any]() ruleStep match { - case SparkSqlStep(name, _, _, MetricPersistType) => { + case SparkSqlStep(name, _, _, MetricPersistType, _) => { try { val pdf = sqlContext.table(s"`${name}`") val records = pdf.toJSON.collect() diff --git a/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/AdaptPhase.scala b/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/AdaptPhase.scala new file mode 100644 index 000000000..d38a4baee --- /dev/null +++ b/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/AdaptPhase.scala @@ -0,0 +1,25 @@ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ +package org.apache.griffin.measure.rules.adaptor + +sealed trait AdaptPhase {} + +final case object PreProcPhase extends AdaptPhase {} + +final case object RunPhase extends AdaptPhase {} \ No newline at end of file diff --git a/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/DataFrameOprAdaptor.scala b/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/DataFrameOprAdaptor.scala index c7219233e..f322b947f 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/DataFrameOprAdaptor.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/DataFrameOprAdaptor.scala @@ -21,14 +21,15 @@ package org.apache.griffin.measure.rules.adaptor import org.apache.griffin.measure.algo.ProcessType import org.apache.griffin.measure.rules.step._ -case class DataFrameOprAdaptor(processType: ProcessType) extends RuleAdaptor { +case class DataFrameOprAdaptor(adaptPhase: AdaptPhase) extends RuleAdaptor { def genRuleStep(param: Map[String, Any]): Seq[RuleStep] = { - DfOprStep(getName(param), getRule(param), getDetails(param), getPersistType(param)) :: Nil + DfOprStep(getName(param), getRule(param), getDetails(param), + getPersistType(param), getUpdateDataSource(param)) :: Nil } def adaptConcreteRuleStep(ruleStep: RuleStep): Seq[ConcreteRuleStep] = { ruleStep match { - case rs @ DfOprStep(_, _, _, _) => rs :: Nil + case rs @ DfOprStep(_, _, _, _, _) => rs :: Nil case _ => Nil } } diff --git a/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/GriffinDslAdaptor.scala b/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/GriffinDslAdaptor.scala index 2b7dd657f..93ca65d1e 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/GriffinDslAdaptor.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/GriffinDslAdaptor.scala @@ -28,14 +28,16 @@ import org.apache.griffin.measure.rules.step._ case class GriffinDslAdaptor(dataSourceNames: Seq[String], functionNames: Seq[String], - processType: ProcessType + adaptPhase: AdaptPhase ) extends RuleAdaptor { object StepInfo { val _Name = "name" val _PersistType = "persist.type" - def getNameOpt(param: Map[String, Any]): Option[String] = param.get(_Name).flatMap(a => Some(a.toString)) + val _UpdateDataSource = "update.data.source" + def getNameOpt(param: Map[String, Any]): Option[String] = param.get(_Name).map(_.toString) def getPersistType(param: Map[String, Any]): PersistType = PersistType(param.getOrElse(_PersistType, "").toString) + def getUpdateDataSourceOpt(param: Map[String, Any]): Option[String] = param.get(_UpdateDataSource).map(_.toString) } object AccuracyInfo { val _Source = "source" @@ -51,7 +53,7 @@ case class GriffinDslAdaptor(dataSourceNames: Seq[String], val _Profiling = "profiling" } - def getNameOpt(param: Map[String, Any], key: String): Option[String] = param.get(key).flatMap(a => Some(a.toString)) + def getNameOpt(param: Map[String, Any], key: String): Option[String] = param.get(key).map(_.toString) def resultName(param: Map[String, Any], key: String): String = { val nameOpt = param.get(key) match { case Some(prm: Map[String, Any]) => StepInfo.getNameOpt(prm) @@ -65,6 +67,12 @@ case class GriffinDslAdaptor(dataSourceNames: Seq[String], case _ => defPersistType } } + def resultUpdateDataSourceOpt(param: Map[String, Any], key: String): Option[String] = { + param.get(key) match { + case Some(prm: Map[String, Any]) => StepInfo.getUpdateDataSourceOpt(prm) + case _ => None + } + } val _dqType = "dq.type" @@ -182,7 +190,8 @@ case class GriffinDslAdaptor(dataSourceNames: Seq[String], missRecordsName, missRecordsSql, Map[String, Any](), - resultPersistType(details, AccuracyInfo._MissRecords, RecordPersistType) + resultPersistType(details, AccuracyInfo._MissRecords, RecordPersistType), + resultUpdateDataSourceOpt(details, AccuracyInfo._MissRecords) ) // 2. miss count @@ -197,13 +206,14 @@ case class GriffinDslAdaptor(dataSourceNames: Seq[String], // } // } val missSql = { - s"SELECT `${GroupByColumn.tmst}`, COUNT(*) AS `${missColName}` FROM `${missRecordsName}` GROUP BY `${GroupByColumn.tmst}`" + s"SELECT `${GroupByColumn.tmst}` AS `${GroupByColumn.tmst}`, COUNT(*) AS `${missColName}` FROM `${missRecordsName}` GROUP BY `${GroupByColumn.tmst}`" } val missStep = SparkSqlStep( missTableName, missSql, Map[String, Any](), - NonePersistType + NonePersistType, + None ) // 3. total count @@ -218,13 +228,14 @@ case class GriffinDslAdaptor(dataSourceNames: Seq[String], // } // } val totalSql = { - s"SELECT `${GroupByColumn.tmst}`, COUNT(*) AS `${totalColName}` FROM `${sourceName}` GROUP BY `${GroupByColumn.tmst}`" + s"SELECT `${GroupByColumn.tmst}` AS `${GroupByColumn.tmst}`, COUNT(*) AS `${totalColName}` FROM `${sourceName}` GROUP BY `${GroupByColumn.tmst}`" } val totalStep = SparkSqlStep( totalTableName, totalSql, Map[String, Any](), - NonePersistType + NonePersistType, + None ) // 4. accuracy metric @@ -276,7 +287,8 @@ case class GriffinDslAdaptor(dataSourceNames: Seq[String], accuracyMetricSql, details, // resultPersistType(details, AccuracyInfo._Accuracy, MetricPersistType) - NonePersistType + NonePersistType, + None ) // 5. accuracy metric filter @@ -290,7 +302,8 @@ case class GriffinDslAdaptor(dataSourceNames: Seq[String], ("matched" -> matchedColName), ("tmst" -> GroupByColumn.tmst) ), - resultPersistType(details, AccuracyInfo._Accuracy, MetricPersistType) + resultPersistType(details, AccuracyInfo._Accuracy, MetricPersistType), + None ) missRecordsStep :: missStep :: totalStep :: accuracyMetricStep :: accuracyStep :: Nil @@ -330,7 +343,8 @@ case class GriffinDslAdaptor(dataSourceNames: Seq[String], profilingMetricName, profilingSql, details, - resultPersistType(details, ProfilingInfo._Profiling, MetricPersistType) + resultPersistType(details, ProfilingInfo._Profiling, MetricPersistType), + None ) profilingStep :: Nil diff --git a/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/RuleAdaptor.scala b/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/RuleAdaptor.scala index 0c59258c8..20f1346b1 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/RuleAdaptor.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/RuleAdaptor.scala @@ -30,16 +30,18 @@ import org.apache.griffin.measure.rules.dsl.{DslType, PersistType} trait RuleAdaptor extends Loggable with Serializable { - val processType: ProcessType + val adaptPhase: AdaptPhase val _name = "name" val _rule = "rule" val _persistType = "persist.type" + val _updateDataSource = "update.data.source" val _details = "details" protected def getName(param: Map[String, Any]) = param.getOrElse(_name, RuleStepNameGenerator.genName).toString protected def getRule(param: Map[String, Any]) = param.getOrElse(_rule, "").toString protected def getPersistType(param: Map[String, Any]) = PersistType(param.getOrElse(_persistType, "").toString) + protected def getUpdateDataSource(param: Map[String, Any]) = param.get(_updateDataSource).map(_.toString) protected def getDetails(param: Map[String, Any]) = param.get(_details) match { case Some(dt: Map[String, Any]) => dt case _ => Map[String, Any]() diff --git a/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/RuleAdaptorGroup.scala b/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/RuleAdaptorGroup.scala index 0e07c742e..cbf065ab6 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/RuleAdaptorGroup.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/RuleAdaptorGroup.scala @@ -47,11 +47,11 @@ object RuleAdaptorGroup { } } - private def genRuleAdaptor(dslType: DslType, dsNames: Seq[String], procType: ProcessType): Option[RuleAdaptor] = { + private def genRuleAdaptor(dslType: DslType, dsNames: Seq[String], adaptPhase: AdaptPhase): Option[RuleAdaptor] = { dslType match { - case SparkSqlType => Some(SparkSqlAdaptor(procType)) - case DfOprType => Some(DataFrameOprAdaptor(procType)) - case GriffinDslType => Some(GriffinDslAdaptor(dsNames, functionNames, procType)) + case SparkSqlType => Some(SparkSqlAdaptor(adaptPhase)) + case DfOprType => Some(DataFrameOprAdaptor(adaptPhase)) + case GriffinDslType => Some(GriffinDslAdaptor(dsNames, functionNames, adaptPhase)) case _ => None } } @@ -72,18 +72,18 @@ object RuleAdaptorGroup { // steps // } - def genConcreteRuleSteps(evaluateRuleParam: EvaluateRuleParam, procType: ProcessType): Seq[ConcreteRuleStep] = { + def genConcreteRuleSteps(evaluateRuleParam: EvaluateRuleParam, adaptPhase: AdaptPhase): Seq[ConcreteRuleStep] = { val dslTypeStr = if (evaluateRuleParam.dslType == null) "" else evaluateRuleParam.dslType val defaultDslType = DslType(dslTypeStr) val ruleParams = evaluateRuleParam.rules - genConcreteRuleSteps(ruleParams, defaultDslType, procType) + genConcreteRuleSteps(ruleParams, defaultDslType, adaptPhase) } - def genConcreteRuleSteps(ruleParams: Seq[Map[String, Any]], defDslType: DslType, procType: ProcessType): Seq[ConcreteRuleStep] = { + def genConcreteRuleSteps(ruleParams: Seq[Map[String, Any]], defDslType: DslType, adaptPhase: AdaptPhase): Seq[ConcreteRuleStep] = { val (steps, dsNames) = ruleParams.foldLeft((Seq[ConcreteRuleStep](), dataSourceNames)) { (res, param) => val (preSteps, preNames) = res val dslType = getDslType(param, defDslType) - val (curSteps, curNames) = genRuleAdaptor(dslType, preNames, procType) match { + val (curSteps, curNames) = genRuleAdaptor(dslType, preNames, adaptPhase) match { case Some(ruleAdaptor) => (ruleAdaptor.genConcreteRuleStep(param), preNames ++ ruleAdaptor.getTempSourceNames(param)) case _ => (Nil, preNames) } diff --git a/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/SparkSqlAdaptor.scala b/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/SparkSqlAdaptor.scala index f8032c461..643ef2d9b 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/SparkSqlAdaptor.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/SparkSqlAdaptor.scala @@ -22,20 +22,21 @@ import org.apache.griffin.measure.algo._ import org.apache.griffin.measure.data.connector.GroupByColumn import org.apache.griffin.measure.rules.step._ -case class SparkSqlAdaptor(processType: ProcessType) extends RuleAdaptor { +case class SparkSqlAdaptor(adaptPhase: AdaptPhase) extends RuleAdaptor { def genRuleStep(param: Map[String, Any]): Seq[RuleStep] = { - SparkSqlStep(getName(param), getRule(param), getDetails(param), getPersistType(param)) :: Nil + SparkSqlStep(getName(param), getRule(param), getDetails(param), + getPersistType(param), getUpdateDataSource(param)) :: Nil } def adaptConcreteRuleStep(ruleStep: RuleStep): Seq[ConcreteRuleStep] = { ruleStep match { - case rs @ SparkSqlStep(name, rule, details, persistType) => { - processType match { - case BatchProcessType => rs :: Nil - case StreamingProcessType => { - val repSel = rule.replaceFirst("(?i)select", s"SELECT `${GroupByColumn.tmst}`,") + case rs @ SparkSqlStep(name, rule, details, persistType, udsOpt) => { + adaptPhase match { + case PreProcPhase => rs :: Nil + case RunPhase => { + val repSel = rule.replaceFirst("(?i)select", s"SELECT `${GroupByColumn.tmst}` AS `${GroupByColumn.tmst}`,") val groupbyRule = repSel.concat(s" GROUP BY `${GroupByColumn.tmst}`") - val nrs = SparkSqlStep(name, groupbyRule, details, persistType) + val nrs = SparkSqlStep(name, groupbyRule, details, persistType, udsOpt) nrs :: Nil } } diff --git a/measure/src/main/scala/org/apache/griffin/measure/rules/step/ConcreteRuleStep.scala b/measure/src/main/scala/org/apache/griffin/measure/rules/step/ConcreteRuleStep.scala index 17fdb4479..f9b6b9098 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rules/step/ConcreteRuleStep.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rules/step/ConcreteRuleStep.scala @@ -24,6 +24,8 @@ trait ConcreteRuleStep extends RuleStep { val persistType: PersistType + val updateDataSource: Option[String] + // def isGroupMetric: Boolean = { // val _GroupMetric = "group.metric" // details.get(_GroupMetric) match { diff --git a/measure/src/main/scala/org/apache/griffin/measure/rules/step/DfOprStep.scala b/measure/src/main/scala/org/apache/griffin/measure/rules/step/DfOprStep.scala index a1162d819..5a85ea91c 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rules/step/DfOprStep.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rules/step/DfOprStep.scala @@ -20,7 +20,8 @@ package org.apache.griffin.measure.rules.step import org.apache.griffin.measure.rules.dsl._ -case class DfOprStep(name: String, rule: String, details: Map[String, Any], persistType: PersistType +case class DfOprStep(name: String, rule: String, details: Map[String, Any], + persistType: PersistType, updateDataSource: Option[String] ) extends ConcreteRuleStep { val dslType: DslType = DfOprType diff --git a/measure/src/main/scala/org/apache/griffin/measure/rules/step/GriffinDslStep.scala b/measure/src/main/scala/org/apache/griffin/measure/rules/step/GriffinDslStep.scala index 057b18b33..68c2c9a1c 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rules/step/GriffinDslStep.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rules/step/GriffinDslStep.scala @@ -20,7 +20,8 @@ package org.apache.griffin.measure.rules.step import org.apache.griffin.measure.rules.dsl._ -case class GriffinDslStep(name: String, rule: String, dqType: DqType, details: Map[String, Any]) extends RuleStep { +case class GriffinDslStep(name: String, rule: String, dqType: DqType, details: Map[String, Any] + ) extends RuleStep { val dslType: DslType = GriffinDslType diff --git a/measure/src/main/scala/org/apache/griffin/measure/rules/step/SparkSqlStep.scala b/measure/src/main/scala/org/apache/griffin/measure/rules/step/SparkSqlStep.scala index 602492b60..943cce55e 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rules/step/SparkSqlStep.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rules/step/SparkSqlStep.scala @@ -21,7 +21,8 @@ package org.apache.griffin.measure.rules.step import org.apache.griffin.measure.persist._ import org.apache.griffin.measure.rules.dsl._ -case class SparkSqlStep(name: String, rule: String, details: Map[String, Any], persistType: PersistType +case class SparkSqlStep(name: String, rule: String, details: Map[String, Any], + persistType: PersistType, updateDataSource: Option[String] ) extends ConcreteRuleStep { val dslType: DslType = SparkSqlType diff --git a/measure/src/test/resources/config-test-accuracy-streaming.json b/measure/src/test/resources/config-test-accuracy-streaming.json index 95c3ee12b..276f8dd70 100644 --- a/measure/src/test/resources/config-test-accuracy-streaming.json +++ b/measure/src/test/resources/config-test-accuracy-streaming.json @@ -102,7 +102,8 @@ "target": "target", "miss.records": { "name": "miss.records", - "persist.type": "record" + "persist.type": "record", + "update.data.source": "source" }, "accuracy": { "name": "accu", From 4113c885c081c8ff0d8f23dc98292939fc8f5281 Mon Sep 17 00:00:00 2001 From: Lionel Liu Date: Thu, 14 Sep 2017 10:14:25 +0800 Subject: [PATCH 070/111] v1 --- .../measure/data/source/DataSource.scala | 5 ++++ .../measure/process/engine/DqEngines.scala | 23 ++++++++++++++----- 2 files changed, 22 insertions(+), 6 deletions(-) diff --git a/measure/src/main/scala/org/apache/griffin/measure/data/source/DataSource.scala b/measure/src/main/scala/org/apache/griffin/measure/data/source/DataSource.scala index 921690615..9cc14f9a0 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/data/source/DataSource.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/data/source/DataSource.scala @@ -22,6 +22,7 @@ import org.apache.griffin.measure.data.connector._ import org.apache.griffin.measure.data.connector.batch._ import org.apache.griffin.measure.data.connector.streaming._ import org.apache.griffin.measure.log.Loggable +import org.apache.spark.rdd.RDD import org.apache.spark.sql.{DataFrame, SQLContext} case class DataSource(name: String, @@ -64,4 +65,8 @@ case class DataSource(name: String, } } + def updateData(rdd: Option[DataFrame], ms: Long): Unit = { + // fixme + } + } diff --git a/measure/src/main/scala/org/apache/griffin/measure/process/engine/DqEngines.scala b/measure/src/main/scala/org/apache/griffin/measure/process/engine/DqEngines.scala index b6a40c73f..d1da7406a 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/process/engine/DqEngines.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/process/engine/DqEngines.scala @@ -90,12 +90,23 @@ case class DqEngines(engines: Seq[DqEngine]) extends DqEngine { // val recordSteps = ruleSteps.filter(_.persistType == RecordPersistType) // recordSteps.foreach { step => // val name = step.name -// val records = collectRecords(step, timeGroups) -// records.foreach { pair => -// val (t, recs) = pair -// dataSources -// val persist = persistFactory.getPersists(t) -// persist.persistRecords(recs, name) +// val udpateDataSources = dataSources.filter { ds => +// step.updateDataSource match { +// case Some(dsName) if (dsName == ds.name) => true +// case _ => false +// } +// } +// if (udpateDataSources.size > 0) { +// val records = collectRecords(step, timeGroups) +// +// records.foreach { pair => +// val (t, recs) = pair +// udpateDataSources.foreach { ds => +// ds.updateData(recs, t) +// } +//// val persist = persistFactory.getPersists(t) +//// persist.persistRecords(recs, name) +// } // } // } } From 0f70c53efb8b4cf8da7ded87857599d9bdcff260 Mon Sep 17 00:00:00 2001 From: Lionel Liu Date: Thu, 14 Sep 2017 10:25:49 +0800 Subject: [PATCH 071/111] v2 --- .../apache/griffin/measure/persist/HdfsPersist.scala | 4 +++- .../apache/griffin/measure/persist/HttpPersist.scala | 3 ++- .../apache/griffin/measure/persist/LoggerPersist.scala | 4 +++- .../apache/griffin/measure/persist/MultiPersists.scala | 3 ++- .../org/apache/griffin/measure/persist/Persist.scala | 3 ++- .../measure/process/engine/DataFrameOprEngine.scala | 10 +++++----- .../griffin/measure/process/engine/DqEngine.scala | 2 +- .../griffin/measure/process/engine/DqEngines.scala | 5 +++-- .../measure/process/engine/SparkSqlEngine.scala | 10 +++++----- 9 files changed, 26 insertions(+), 18 deletions(-) diff --git a/measure/src/main/scala/org/apache/griffin/measure/persist/HdfsPersist.scala b/measure/src/main/scala/org/apache/griffin/measure/persist/HdfsPersist.scala index 23990cfb9..200ef8424 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/persist/HdfsPersist.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/persist/HdfsPersist.scala @@ -23,6 +23,7 @@ import java.util.Date import org.apache.griffin.measure.result._ import org.apache.griffin.measure.utils.{HdfsUtil, JsonUtil} import org.apache.spark.rdd.RDD +import org.apache.spark.sql.DataFrame import scala.util.Try @@ -196,7 +197,8 @@ case class HdfsPersist(config: Map[String, Any], metricName: String, timeStamp: } - def persistRecords(records: RDD[String], name: String): Unit = { + def persistRecords(df: DataFrame, name: String): Unit = { + val records = df.toJSON val path = filePath(name) try { val recordCount = records.count diff --git a/measure/src/main/scala/org/apache/griffin/measure/persist/HttpPersist.scala b/measure/src/main/scala/org/apache/griffin/measure/persist/HttpPersist.scala index a15331c1a..6d5e9a06b 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/persist/HttpPersist.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/persist/HttpPersist.scala @@ -21,6 +21,7 @@ package org.apache.griffin.measure.persist import org.apache.griffin.measure.result._ import org.apache.griffin.measure.utils.{HttpUtil, JsonUtil} import org.apache.spark.rdd.RDD +import org.apache.spark.sql.DataFrame import scala.util.Try @@ -87,7 +88,7 @@ case class HttpPersist(config: Map[String, Any], metricName: String, timeStamp: def log(rt: Long, msg: String): Unit = {} - def persistRecords(records: RDD[String], name: String): Unit = {} + def persistRecords(df: DataFrame, name: String): Unit = {} // def persistMetrics(metrics: Seq[String], name: String): Unit = { // val maps = metrics.flatMap { m => diff --git a/measure/src/main/scala/org/apache/griffin/measure/persist/LoggerPersist.scala b/measure/src/main/scala/org/apache/griffin/measure/persist/LoggerPersist.scala index b89d3a024..5b2be3e7c 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/persist/LoggerPersist.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/persist/LoggerPersist.scala @@ -23,6 +23,7 @@ import java.util.Date import org.apache.griffin.measure.result._ import org.apache.griffin.measure.utils.{HdfsUtil, JsonUtil} import org.apache.spark.rdd.RDD +import org.apache.spark.sql.DataFrame // persist result and data to hdfs case class LoggerPersist(config: Map[String, Any], metricName: String, timeStamp: Long) extends Persist { @@ -115,7 +116,8 @@ case class LoggerPersist(config: Map[String, Any], metricName: String, timeStamp println(s"[${timeStamp}] ${rt}: ${msg}") } - def persistRecords(records: RDD[String], name: String): Unit = { + def persistRecords(df: DataFrame, name: String): Unit = { + val records = df.toJSON println(s"${name} [${timeStamp}] records: ") try { val recordCount = records.count.toInt diff --git a/measure/src/main/scala/org/apache/griffin/measure/persist/MultiPersists.scala b/measure/src/main/scala/org/apache/griffin/measure/persist/MultiPersists.scala index 90550842e..f14f9cbc5 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/persist/MultiPersists.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/persist/MultiPersists.scala @@ -21,6 +21,7 @@ package org.apache.griffin.measure.persist import org.apache.griffin.measure.result._ import org.apache.griffin.measure.utils.{HttpUtil, JsonUtil} import org.apache.spark.rdd.RDD +import org.apache.spark.sql.DataFrame import scala.util.Try @@ -49,7 +50,7 @@ case class MultiPersists(persists: Iterable[Persist]) extends Persist { def log(rt: Long, msg: String): Unit = { persists.foreach(_.log(rt, msg)) } - def persistRecords(records: RDD[String], name: String): Unit = { persists.foreach(_.persistRecords(records, name)) } + def persistRecords(df: DataFrame, name: String): Unit = { persists.foreach(_.persistRecords(df, name)) } // def persistMetrics(metrics: Seq[String], name: String): Unit = { persists.foreach(_.persistMetrics(metrics, name)) } def persistMetrics(metrics: Map[String, Any]): Unit = { persists.foreach(_.persistMetrics(metrics)) } diff --git a/measure/src/main/scala/org/apache/griffin/measure/persist/Persist.scala b/measure/src/main/scala/org/apache/griffin/measure/persist/Persist.scala index 0de763618..4c73a2b79 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/persist/Persist.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/persist/Persist.scala @@ -21,6 +21,7 @@ package org.apache.griffin.measure.persist import org.apache.griffin.measure.log.Loggable import org.apache.griffin.measure.result._ import org.apache.spark.rdd.RDD +import org.apache.spark.sql.DataFrame import scala.util.Try @@ -42,7 +43,7 @@ trait Persist extends Loggable with Serializable { // def records(recs: RDD[String], tp: String): Unit // def records(recs: Iterable[String], tp: String): Unit - def persistRecords(records: RDD[String], name: String): Unit + def persistRecords(df: DataFrame, name: String): Unit // def persistMetrics(metrics: Seq[String], name: String): Unit def persistMetrics(metrics: Map[String, Any]): Unit diff --git a/measure/src/main/scala/org/apache/griffin/measure/process/engine/DataFrameOprEngine.scala b/measure/src/main/scala/org/apache/griffin/measure/process/engine/DataFrameOprEngine.scala index f49da08c7..3bcc91984 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/process/engine/DataFrameOprEngine.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/process/engine/DataFrameOprEngine.scala @@ -96,15 +96,15 @@ case class DataFrameOprEngine(sqlContext: SQLContext, @transient ssc: StreamingC // true // } - def collectRecords(ruleStep: ConcreteRuleStep, timeGroups: Iterable[Long]): Map[Long, RDD[String]] = { + def collectRecords(ruleStep: ConcreteRuleStep, timeGroups: Iterable[Long]): Map[Long, DataFrame] = { ruleStep match { case DfOprStep(name, _, _, RecordPersistType, _) => { try { val pdf = sqlContext.table(s"`${name}`") timeGroups.flatMap { timeGroup => try { - val rdd = pdf.filter(s"`${GroupByColumn.tmst}` = ${timeGroup}").toJSON - Some((timeGroup, rdd)) + val tdf = pdf.filter(s"`${GroupByColumn.tmst}` = ${timeGroup}") + Some((timeGroup, tdf)) } catch { case e: Throwable => None } @@ -112,11 +112,11 @@ case class DataFrameOprEngine(sqlContext: SQLContext, @transient ssc: StreamingC } catch { case e: Throwable => { error(s"persist result ${name} error: ${e.getMessage}") - Map[Long, RDD[String]]() + Map[Long, DataFrame]() } } } - case _ => Map[Long, RDD[String]]() + case _ => Map[Long, DataFrame]() } } diff --git a/measure/src/main/scala/org/apache/griffin/measure/process/engine/DqEngine.scala b/measure/src/main/scala/org/apache/griffin/measure/process/engine/DqEngine.scala index 24a97d682..3c63756dc 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/process/engine/DqEngine.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/process/engine/DqEngine.scala @@ -39,5 +39,5 @@ trait DqEngine extends Loggable with Serializable { def collectMetrics(ruleStep: ConcreteRuleStep): Map[Long, Map[String, Any]] - def collectRecords(ruleStep: ConcreteRuleStep, timeGroups: Iterable[Long]): Map[Long, RDD[String]] + def collectRecords(ruleStep: ConcreteRuleStep, timeGroups: Iterable[Long]): Map[Long, DataFrame] } diff --git a/measure/src/main/scala/org/apache/griffin/measure/process/engine/DqEngines.scala b/measure/src/main/scala/org/apache/griffin/measure/process/engine/DqEngines.scala index d1da7406a..341c8520a 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/process/engine/DqEngines.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/process/engine/DqEngines.scala @@ -26,6 +26,7 @@ import org.apache.griffin.measure.persist.{Persist, PersistFactory} import org.apache.griffin.measure.rules.dsl._ import org.apache.griffin.measure.rules.step._ import org.apache.spark.rdd.RDD +import org.apache.spark.sql.DataFrame case class DqEngines(engines: Seq[DqEngine]) extends DqEngine { @@ -169,8 +170,8 @@ case class DqEngines(engines: Seq[DqEngine]) extends DqEngine { // if (!ret) error(s"persist result warn: no dq engine support ${ruleStep}") // ret // } - def collectRecords(ruleStep: ConcreteRuleStep, timeGroups: Iterable[Long]): Map[Long, RDD[String]] = { - val ret = engines.foldLeft(Map[Long, RDD[String]]()) { (ret, engine) => + def collectRecords(ruleStep: ConcreteRuleStep, timeGroups: Iterable[Long]): Map[Long, DataFrame] = { + val ret = engines.foldLeft(Map[Long, DataFrame]()) { (ret, engine) => ret ++ engine.collectRecords(ruleStep, timeGroups) } // if (ret.isEmpty) warn(s"collect records warn: no records collected for ${ruleStep}") diff --git a/measure/src/main/scala/org/apache/griffin/measure/process/engine/SparkSqlEngine.scala b/measure/src/main/scala/org/apache/griffin/measure/process/engine/SparkSqlEngine.scala index addc422d8..0b16013ad 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/process/engine/SparkSqlEngine.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/process/engine/SparkSqlEngine.scala @@ -85,15 +85,15 @@ case class SparkSqlEngine(sqlContext: SQLContext, @transient ssc: StreamingConte // } // } - def collectRecords(ruleStep: ConcreteRuleStep, timeGroups: Iterable[Long]): Map[Long, RDD[String]] = { + def collectRecords(ruleStep: ConcreteRuleStep, timeGroups: Iterable[Long]): Map[Long, DataFrame] = { ruleStep match { case SparkSqlStep(name, _, _, RecordPersistType, _) => { try { val pdf = sqlContext.table(s"`${name}`") timeGroups.flatMap { timeGroup => try { - val rdd = pdf.filter(s"`${GroupByColumn.tmst}` = ${timeGroup}").toJSON - Some((timeGroup, rdd)) + val tdf = pdf.filter(s"`${GroupByColumn.tmst}` = ${timeGroup}") + Some((timeGroup, tdf)) } catch { case e: Throwable => None } @@ -101,11 +101,11 @@ case class SparkSqlEngine(sqlContext: SQLContext, @transient ssc: StreamingConte } catch { case e: Throwable => { error(s"persist result ${name} error: ${e.getMessage}") - Map[Long, RDD[String]]() + Map[Long, DataFrame]() } } } - case _ => Map[Long, RDD[String]]() + case _ => Map[Long, DataFrame]() } } From 4b0e455341adbf169041d0a1f7f2146595e6684b Mon Sep 17 00:00:00 2001 From: Lionel Liu Date: Thu, 14 Sep 2017 14:44:57 +0800 Subject: [PATCH 072/111] init --- .../measure/data/source/DataSource.scala | 40 ++++++++-- .../measure/data/source/DataSourceCache.scala | 80 +++++++++++++------ .../data/source/DataSourceFactory.scala | 2 +- .../measure/process/StreamingDqThread.scala | 10 ++- .../process/engine/DataFrameOprEngine.scala | 1 + .../measure/process/engine/DqEngines.scala | 55 ++++++------- 6 files changed, 125 insertions(+), 63 deletions(-) diff --git a/measure/src/main/scala/org/apache/griffin/measure/data/source/DataSource.scala b/measure/src/main/scala/org/apache/griffin/measure/data/source/DataSource.scala index 9cc14f9a0..5cf25823b 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/data/source/DataSource.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/data/source/DataSource.scala @@ -25,7 +25,8 @@ import org.apache.griffin.measure.log.Loggable import org.apache.spark.rdd.RDD import org.apache.spark.sql.{DataFrame, SQLContext} -case class DataSource(name: String, +case class DataSource(sqlContext: SQLContext, + name: String, dataConnectors: Seq[DataConnector], dataSourceCacheOpt: Option[DataSourceCache] ) extends Loggable with Serializable { @@ -39,34 +40,59 @@ case class DataSource(name: String, dataConnectors.foreach(_.init) } - def loadData(ms: Long): Unit = { + def loadData(ms: Long): Boolean = { data(ms) match { case Some(df) => { df.registerTempTable(name) + true } case None => { - throw new Exception(s"load data source [${name}] fails") +// val df = sqlContext.emptyDataFrame +// df.registerTempTable(name) + warn(s"load data source [${name}] fails") + false +// throw new Exception(s"load data source [${name}] fails") } } } + def dropTable(): Unit = { + try { + sqlContext.dropTempTable(name) + } catch { + case e: Throwable => warn(s"drop table [${name}] fails") + } + } + private def data(ms: Long): Option[DataFrame] = { val batchDataFrameOpt = batchDataConnectors.flatMap { dc => dc.data(ms) - }.reduceOption(_ unionAll _) + }.reduceOption((a, b) => unionDataFrames(a, b)) val cacheDataFrameOpt = dataSourceCacheOpt.flatMap(_.readData()) (batchDataFrameOpt, cacheDataFrameOpt) match { - case (Some(bdf), Some(cdf)) => Some(bdf unionAll cdf) + case (Some(bdf), Some(cdf)) => Some(unionDataFrames(bdf, cdf)) case (Some(bdf), _) => Some(bdf) case (_, Some(cdf)) => Some(cdf) case _ => None } } - def updateData(rdd: Option[DataFrame], ms: Long): Unit = { - // fixme + private def unionDataFrames(df1: DataFrame, df2: DataFrame): DataFrame = { + try { + df1 unionAll df2 + } catch { + case e: Throwable => df1 + } + } + + def updateData(df: DataFrame, ms: Long): Unit = { + dataSourceCacheOpt.foreach(_.updateData(df, ms)) + } + + def cleanOldData(): Unit = { + dataSourceCacheOpt.foreach(_.cleanOldData) } } diff --git a/measure/src/main/scala/org/apache/griffin/measure/data/source/DataSourceCache.scala b/measure/src/main/scala/org/apache/griffin/measure/data/source/DataSourceCache.scala index 953b30789..0fb5205df 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/data/source/DataSourceCache.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/data/source/DataSourceCache.scala @@ -131,36 +131,68 @@ case class DataSourceCache(sqlContext: SQLContext, param: Map[String, Any], Some(sqlContext.read.json(partitionPaths: _*)) } catch { case e: Throwable => { - error(s"read data source cache error: ${e.getMessage}") + warn(s"read data source cache warn: ${e.getMessage}") None } } } } - def updateOldData(t: Long, oldData: Iterable[Map[String, Any]]): Unit = { - // fixme - // parallel process different time groups, lock is unnecessary -// val ptns = getPartition(t) -// val ptnsPath = genPartitionHdfsPath(ptns) -// val dirPath = s"${filePath}/${ptnsPath}" -// val dataFileName = s"${t}" -// val dataFilePath = HdfsUtil.getHdfsFilePath(dirPath, dataFileName) -// -// try { -// // remove out time old data -// HdfsFileDumpUtil.remove(dirPath, dataFileName, true) -// -// // save updated old data -// if (oldData.size > 0) { -// val recordDatas = oldData.flatMap { dt => -// encode(dt, t) -// } -// val dumped = HdfsFileDumpUtil.dump(dataFilePath, recordDatas, rowSepLiteral) -// } -// } catch { -// case e: Throwable => error(s"update old data error: ${e.getMessage}") -// } + def updateData(df: DataFrame, ms: Long): Unit = { + val ptns = getPartition(ms) + val ptnsPath = genPartitionHdfsPath(ptns) + val dirPath = s"${filePath}/${ptnsPath}" + val dataFileName = s"${ms}" + val dataFilePath = HdfsUtil.getHdfsFilePath(dirPath, dataFileName) + + try { + val records = df.toJSON + val arr = records.collect + val needSave = !arr.isEmpty + + // remove out time old data + HdfsFileDumpUtil.remove(dirPath, dataFileName, true) + println(s"remove file path: ${dirPath}/${dataFileName}") + + // save updated data + val dumped = if (needSave) { + HdfsFileDumpUtil.dump(dataFilePath, arr, rowSepLiteral) + println(s"update file path: ${dataFilePath}") + } else false + } catch { + case e: Throwable => error(s"update data error: ${e.getMessage}") + } + } + + def cleanOldData(): Unit = { + val oldCacheLocked = oldCacheLock.lock(-1, TimeUnit.SECONDS) + if (oldCacheLocked) { + try { + val cleanTime = readCleanTime() + cleanTime match { + case Some(ct) => { + // drop partitions + val bounds = getPartition(ct) + + // list partition paths + val earlierPaths = listPathsEarlierThanBounds(filePath :: Nil, bounds) + + // delete out time data path + earlierPaths.foreach { path => + println(s"delete hdfs path: ${path}") + HdfsUtil.deleteHdfsPath(path) + } + } + case _ => { + // do nothing + } + } + } catch { + case e: Throwable => error(s"clean old data error: ${e.getMessage}") + } finally { + oldCacheLock.unlock() + } + } } override protected def genCleanTime(ms: Long): Long = { diff --git a/measure/src/main/scala/org/apache/griffin/measure/data/source/DataSourceFactory.scala b/measure/src/main/scala/org/apache/griffin/measure/data/source/DataSourceFactory.scala index a713f3caf..6c1b76eec 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/data/source/DataSourceFactory.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/data/source/DataSourceFactory.scala @@ -59,7 +59,7 @@ object DataSourceFactory extends Loggable { } val dataSourceCacheOpt = genDataSourceCache(sqlContext, cacheParam, metricName, index) - Some(DataSource(name, dataConnectors, dataSourceCacheOpt)) + Some(DataSource(sqlContext, name, dataConnectors, dataSourceCacheOpt)) } private def genDataSourceCache(sqlContext: SQLContext, param: Map[String, Any], diff --git a/measure/src/main/scala/org/apache/griffin/measure/process/StreamingDqThread.scala b/measure/src/main/scala/org/apache/griffin/measure/process/StreamingDqThread.scala index 1f19840a7..b83cb4ab0 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/process/StreamingDqThread.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/process/StreamingDqThread.scala @@ -65,11 +65,13 @@ case class StreamingDqThread(dqEngines: DqEngines, val ct = new Date().getTime appPersist.log(ct, s"calculation using time: ${ct - st} ms") - // persist results and cache records + // persist results val timeGroups = dqEngines.persistAllMetrics(ruleSteps, persistFactory) - dqEngines.updateDataSources(ruleSteps, dataSources, timeGroups) dqEngines.persistAllRecords(ruleSteps, persistFactory, timeGroups) + // update data source + dqEngines.updateDataSources(ruleSteps, dataSources, timeGroups) + TimeInfoCache.endTimeInfoCache // clean old data @@ -93,8 +95,8 @@ case class StreamingDqThread(dqEngines: DqEngines, // clean old data and old result cache private def cleanData(): Unit = { try { -// sourceDataConnector.cleanOldData -// targetDataConnector.cleanOldData + dataSources.foreach(_.cleanOldData) + dataSources.foreach(_.dropTable) val cleanTime = TimeInfoCache.getCleanTime CacheResultProcesser.refresh(cleanTime) diff --git a/measure/src/main/scala/org/apache/griffin/measure/process/engine/DataFrameOprEngine.scala b/measure/src/main/scala/org/apache/griffin/measure/process/engine/DataFrameOprEngine.scala index 3bcc91984..fbe5b6b97 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/process/engine/DataFrameOprEngine.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/process/engine/DataFrameOprEngine.scala @@ -52,6 +52,7 @@ case class DataFrameOprEngine(sqlContext: SQLContext, @transient ssc: StreamingC } case DataFrameOprs._accuracy => { val df = DataFrameOprs.accuracy(sqlContext, details) + df.show(10) df.registerTempTable(name) } case _ => { diff --git a/measure/src/main/scala/org/apache/griffin/measure/process/engine/DqEngines.scala b/measure/src/main/scala/org/apache/griffin/measure/process/engine/DqEngines.scala index 341c8520a..624f52c87 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/process/engine/DqEngines.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/process/engine/DqEngines.scala @@ -32,13 +32,17 @@ case class DqEngines(engines: Seq[DqEngine]) extends DqEngine { val persistOrder: List[PersistType] = List(MetricPersistType, RecordPersistType) - def loadData(dataSources: Seq[DataSource], ms: Long): Unit = { + def loadData(dataSources: Seq[DataSource], ms: Long): Map[String, Boolean] = { // val dataSources = dataSourceParams.flatMap { param => // genDataSource(param) // } - dataSources.foreach { ds => - ds.loadData(ms) - } +// dataSources.foreach { ds => +// ds.loadData(ms) +// } + dataSources.map { ds => + val load = ds.loadData(ms) + (ds.name, load) + }.toMap } def runRuleSteps(ruleSteps: Seq[ConcreteRuleStep]): Unit = { @@ -87,29 +91,26 @@ case class DqEngines(engines: Seq[DqEngine]) extends DqEngine { def updateDataSources(ruleSteps: Seq[ConcreteRuleStep], dataSources: Seq[DataSource], timeGroups: Iterable[Long]): Unit = { - // fixme -// val recordSteps = ruleSteps.filter(_.persistType == RecordPersistType) -// recordSteps.foreach { step => -// val name = step.name -// val udpateDataSources = dataSources.filter { ds => -// step.updateDataSource match { -// case Some(dsName) if (dsName == ds.name) => true -// case _ => false -// } -// } -// if (udpateDataSources.size > 0) { -// val records = collectRecords(step, timeGroups) -// -// records.foreach { pair => -// val (t, recs) = pair -// udpateDataSources.foreach { ds => -// ds.updateData(recs, t) -// } -//// val persist = persistFactory.getPersists(t) -//// persist.persistRecords(recs, name) -// } -// } -// } + val recordSteps = ruleSteps.filter(_.persistType == RecordPersistType) + recordSteps.foreach { step => + val name = step.name + val udpateDataSources = dataSources.filter { ds => + step.updateDataSource match { + case Some(dsName) if (dsName == ds.name) => true + case _ => false + } + } + if (udpateDataSources.size > 0) { + val records = collectRecords(step, timeGroups) + + records.foreach { pair => + val (t, recs) = pair + udpateDataSources.foreach { ds => + ds.updateData(recs, t) + } + } + } + } } // def persistAllResults(ruleSteps: Seq[ConcreteRuleStep], persistFactory: PersistFactory): Unit = { From 402e73fd17be5458472e51c45b2e486ef5ca6062 Mon Sep 17 00:00:00 2001 From: Lionel Liu Date: Thu, 14 Sep 2017 15:58:53 +0800 Subject: [PATCH 073/111] data checker --- .../measure/data/source/DataSource.scala | 4 +- .../measure/process/check/DataChecker.scala | 29 ++ .../process/engine/DataFrameOprEngine.scala | 1 - .../measure/process/engine/DqEngines.scala | 14 +- .../rules/adaptor/GriffinDslAdaptor.scala | 271 +++++++----------- .../rules/adaptor/RuleAdaptorGroup.scala | 13 +- 6 files changed, 159 insertions(+), 173 deletions(-) create mode 100644 measure/src/main/scala/org/apache/griffin/measure/process/check/DataChecker.scala diff --git a/measure/src/main/scala/org/apache/griffin/measure/data/source/DataSource.scala b/measure/src/main/scala/org/apache/griffin/measure/data/source/DataSource.scala index 5cf25823b..93dc84b9a 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/data/source/DataSource.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/data/source/DataSource.scala @@ -40,17 +40,15 @@ case class DataSource(sqlContext: SQLContext, dataConnectors.foreach(_.init) } - def loadData(ms: Long): Boolean = { + def loadData(ms: Long): Unit = { data(ms) match { case Some(df) => { df.registerTempTable(name) - true } case None => { // val df = sqlContext.emptyDataFrame // df.registerTempTable(name) warn(s"load data source [${name}] fails") - false // throw new Exception(s"load data source [${name}] fails") } } diff --git a/measure/src/main/scala/org/apache/griffin/measure/process/check/DataChecker.scala b/measure/src/main/scala/org/apache/griffin/measure/process/check/DataChecker.scala new file mode 100644 index 000000000..91855c22e --- /dev/null +++ b/measure/src/main/scala/org/apache/griffin/measure/process/check/DataChecker.scala @@ -0,0 +1,29 @@ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ +package org.apache.griffin.measure.process.check + +import org.apache.spark.sql.SQLContext + +case class DataChecker(sqlContext: SQLContext) { + + def existDataSourceName(name: String): Boolean = { + sqlContext.tableNames.exists(_ == name) + } + +} diff --git a/measure/src/main/scala/org/apache/griffin/measure/process/engine/DataFrameOprEngine.scala b/measure/src/main/scala/org/apache/griffin/measure/process/engine/DataFrameOprEngine.scala index fbe5b6b97..3bcc91984 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/process/engine/DataFrameOprEngine.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/process/engine/DataFrameOprEngine.scala @@ -52,7 +52,6 @@ case class DataFrameOprEngine(sqlContext: SQLContext, @transient ssc: StreamingC } case DataFrameOprs._accuracy => { val df = DataFrameOprs.accuracy(sqlContext, details) - df.show(10) df.registerTempTable(name) } case _ => { diff --git a/measure/src/main/scala/org/apache/griffin/measure/process/engine/DqEngines.scala b/measure/src/main/scala/org/apache/griffin/measure/process/engine/DqEngines.scala index 624f52c87..4113ca24a 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/process/engine/DqEngines.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/process/engine/DqEngines.scala @@ -32,17 +32,17 @@ case class DqEngines(engines: Seq[DqEngine]) extends DqEngine { val persistOrder: List[PersistType] = List(MetricPersistType, RecordPersistType) - def loadData(dataSources: Seq[DataSource], ms: Long): Map[String, Boolean] = { + def loadData(dataSources: Seq[DataSource], ms: Long): Unit = { // val dataSources = dataSourceParams.flatMap { param => // genDataSource(param) // } -// dataSources.foreach { ds => -// ds.loadData(ms) + dataSources.foreach { ds => + ds.loadData(ms) + } +// dataSources.flatMap { ds => +// val success = ds.loadData(ms) +// if (success) Some(ds.name) else None // } - dataSources.map { ds => - val load = ds.loadData(ms) - (ds.name, load) - }.toMap } def runRuleSteps(ruleSteps: Seq[ConcreteRuleStep]): Unit = { diff --git a/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/GriffinDslAdaptor.scala b/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/GriffinDslAdaptor.scala index 93ca65d1e..b2e9729f8 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/GriffinDslAdaptor.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/GriffinDslAdaptor.scala @@ -148,7 +148,8 @@ case class GriffinDslAdaptor(dataSourceNames: Seq[String], } } - private def transConcreteRuleSteps(ruleStep: GriffinDslStep, expr: Expr): Seq[ConcreteRuleStep] = { + private def transConcreteRuleSteps(ruleStep: GriffinDslStep, expr: Expr + ): Seq[ConcreteRuleStep] = { val details = ruleStep.details ruleStep.dqType match { case AccuracyType => { @@ -162,151 +163,101 @@ case class GriffinDslAdaptor(dataSourceNames: Seq[String], } val analyzer = AccuracyAnalyzer(expr.asInstanceOf[LogicalExpr], sourceName, targetName) - // 1. miss record - val missRecordsSql = { -// val selClause = analyzer.selectionExprs.map { sel => -// val alias = sel.alias match { -// case Some(a) => s" AS ${a}" -// case _ => "" -// } -// s"${sel.desc}${alias}" -// }.mkString(", ") - val selClause = s"`${sourceName}`.*" - val onClause = expr.coalesceDesc - - val sourceIsNull = analyzer.sourceSelectionExprs.map { sel => - s"${sel.desc} IS NULL" - }.mkString(" AND ") - val targetIsNull = analyzer.targetSelectionExprs.map { sel => - s"${sel.desc} IS NULL" - }.mkString(" AND ") - val whereClause = s"(NOT (${sourceIsNull})) AND (${targetIsNull})" - - s"SELECT ${selClause} FROM `${sourceName}` LEFT JOIN `${targetName}` ON ${onClause} WHERE ${whereClause}" - } - val missRecordsName = resultName(details, AccuracyInfo._MissRecords) - val missRecordsStep = SparkSqlStep( - missRecordsName, - missRecordsSql, - Map[String, Any](), - resultPersistType(details, AccuracyInfo._MissRecords, RecordPersistType), - resultUpdateDataSourceOpt(details, AccuracyInfo._MissRecords) - ) - - // 2. miss count - val missTableName = "_miss_" - val missColName = getNameOpt(details, AccuracyInfo._Miss).getOrElse(AccuracyInfo._Miss) -// val missSql = processType match { -// case BatchProcessType => { -// s"SELECT COUNT(*) AS `${missColName}` FROM `${missRecordsName}`" -// } -// case StreamingProcessType => { -// s"SELECT `${GroupByColumn.tmst}`, COUNT(*) AS `${missColName}` FROM `${missRecordsName}` GROUP BY `${GroupByColumn.tmst}`" -// } -// } - val missSql = { - s"SELECT `${GroupByColumn.tmst}` AS `${GroupByColumn.tmst}`, COUNT(*) AS `${missColName}` FROM `${missRecordsName}` GROUP BY `${GroupByColumn.tmst}`" - } - val missStep = SparkSqlStep( - missTableName, - missSql, - Map[String, Any](), - NonePersistType, - None - ) - - // 3. total count - val totalTableName = "_total_" - val totalColName = getNameOpt(details, AccuracyInfo._Total).getOrElse(AccuracyInfo._Total) -// val totalSql = processType match { -// case BatchProcessType => { -// s"SELECT COUNT(*) AS `${totalColName}` FROM `${sourceName}`" -// } -// case StreamingProcessType => { -// s"SELECT `${GroupByColumn.tmst}`, COUNT(*) AS `${totalColName}` FROM `${sourceName}` GROUP BY `${GroupByColumn.tmst}`" -// } -// } - val totalSql = { - s"SELECT `${GroupByColumn.tmst}` AS `${GroupByColumn.tmst}`, COUNT(*) AS `${totalColName}` FROM `${sourceName}` GROUP BY `${GroupByColumn.tmst}`" - } - val totalStep = SparkSqlStep( - totalTableName, - totalSql, - Map[String, Any](), - NonePersistType, - None - ) + if (!checkDataSourceExists(sourceName)) { + Nil + } else { + // 1. miss record + val missRecordsSql = if (!checkDataSourceExists(targetName)) { + val selClause = s"`${sourceName}`.*" + s"SELECT ${selClause} FROM `${sourceName}`" + } else { + val selClause = s"`${sourceName}`.*" + val onClause = expr.coalesceDesc + val sourceIsNull = analyzer.sourceSelectionExprs.map { sel => + s"${sel.desc} IS NULL" + }.mkString(" AND ") + val targetIsNull = analyzer.targetSelectionExprs.map { sel => + s"${sel.desc} IS NULL" + }.mkString(" AND ") + val whereClause = s"(NOT (${sourceIsNull})) AND (${targetIsNull})" + s"SELECT ${selClause} FROM `${sourceName}` LEFT JOIN `${targetName}` ON ${onClause} WHERE ${whereClause}" + } + val missRecordsName = resultName(details, AccuracyInfo._MissRecords) + val missRecordsStep = SparkSqlStep( + missRecordsName, + missRecordsSql, + Map[String, Any](), + resultPersistType(details, AccuracyInfo._MissRecords, RecordPersistType), + resultUpdateDataSourceOpt(details, AccuracyInfo._MissRecords) + ) - // 4. accuracy metric - val matchedColName = getNameOpt(details, AccuracyInfo._Matched).getOrElse(AccuracyInfo._Matched) -// val accuracyMetricSql = processType match { -// case BatchProcessType => { -// s""" -// |SELECT `${totalTableName}`.`${totalColName}` AS `${totalColName}`, -// |`${missTableName}`.`${missColName}` AS `${missColName}`, -// |(`${totalColName}` - `${missColName}`) AS `${matchedColName}` -// |FROM `${totalTableName}` JOIN `${missTableName}` -// """.stripMargin -// } -// case StreamingProcessType => { -// s""" -// |SELECT `${missTableName}`.`${GroupByColumn.tmst}`, -// |`${totalTableName}`.`${totalColName}` AS `${totalColName}`, -// |`${missTableName}`.`${missColName}` AS `${missColName}`, -// |(`${totalColName}` - `${missColName}`) AS `${matchedColName}` -// |FROM `${totalTableName}` JOIN `${missTableName}` -// |ON `${totalTableName}`.`${GroupByColumn.tmst}` = `${missTableName}`.`${GroupByColumn.tmst}` -// """.stripMargin -// } -// } + // 2. miss count + val missTableName = "_miss_" + val missColName = getNameOpt(details, AccuracyInfo._Miss).getOrElse(AccuracyInfo._Miss) + val missSql = { + s"SELECT `${GroupByColumn.tmst}` AS `${GroupByColumn.tmst}`, COUNT(*) AS `${missColName}` FROM `${missRecordsName}` GROUP BY `${GroupByColumn.tmst}`" + } + val missStep = SparkSqlStep( + missTableName, + missSql, + Map[String, Any](), + NonePersistType, + None + ) + // 3. total count + val totalTableName = "_total_" + val totalColName = getNameOpt(details, AccuracyInfo._Total).getOrElse(AccuracyInfo._Total) + val totalSql = { + s"SELECT `${GroupByColumn.tmst}` AS `${GroupByColumn.tmst}`, COUNT(*) AS `${totalColName}` FROM `${sourceName}` GROUP BY `${GroupByColumn.tmst}`" + } + val totalStep = SparkSqlStep( + totalTableName, + totalSql, + Map[String, Any](), + NonePersistType, + None + ) -// val accuracyMetricSql = { -// s""" -// |SELECT `${missTableName}`.`${GroupByColumn.tmst}`, -// |`${totalTableName}`.`${totalColName}` AS `${totalColName}`, -// |`${missTableName}`.`${missColName}` AS `${missColName}`, -// |(`${totalColName}` - `${missColName}`) AS `${matchedColName}` -// |FROM `${totalTableName}` JOIN `${missTableName}` -// |ON `${totalTableName}`.`${GroupByColumn.tmst}` = `${missTableName}`.`${GroupByColumn.tmst}` -// """.stripMargin -// } - val accuracyMetricSql = { - s""" - |SELECT `${totalTableName}`.`${GroupByColumn.tmst}` AS `${GroupByColumn.tmst}`, - |`${missTableName}`.`${missColName}` AS `${missColName}`, - |`${totalTableName}`.`${totalColName}` AS `${totalColName}` - |FROM `${totalTableName}` FULL JOIN `${missTableName}` - |ON `${totalTableName}`.`${GroupByColumn.tmst}` = `${missTableName}`.`${GroupByColumn.tmst}` + // 4. accuracy metric + val matchedColName = getNameOpt(details, AccuracyInfo._Matched).getOrElse(AccuracyInfo._Matched) + val accuracyMetricSql = { + s""" + |SELECT `${totalTableName}`.`${GroupByColumn.tmst}` AS `${GroupByColumn.tmst}`, + |`${missTableName}`.`${missColName}` AS `${missColName}`, + |`${totalTableName}`.`${totalColName}` AS `${totalColName}` + |FROM `${totalTableName}` FULL JOIN `${missTableName}` + |ON `${totalTableName}`.`${GroupByColumn.tmst}` = `${missTableName}`.`${GroupByColumn.tmst}` """.stripMargin - } - val accuracyMetricName = resultName(details, AccuracyInfo._Accuracy) - val accuracyMetricStep = SparkSqlStep( - accuracyMetricName, - accuracyMetricSql, - details, -// resultPersistType(details, AccuracyInfo._Accuracy, MetricPersistType) - NonePersistType, - None - ) + } + val accuracyMetricName = resultName(details, AccuracyInfo._Accuracy) + val accuracyMetricStep = SparkSqlStep( + accuracyMetricName, + accuracyMetricSql, + details, + // resultPersistType(details, AccuracyInfo._Accuracy, MetricPersistType) + NonePersistType, + None + ) - // 5. accuracy metric filter - val accuracyStep = DfOprStep( - accuracyMetricName, - "accuracy", - Map[String, Any]( - ("df.name" -> accuracyMetricName), - ("miss" -> missColName), - ("total" -> totalColName), - ("matched" -> matchedColName), - ("tmst" -> GroupByColumn.tmst) - ), - resultPersistType(details, AccuracyInfo._Accuracy, MetricPersistType), - None - ) + // 5. accuracy metric filter + val accuracyStep = DfOprStep( + accuracyMetricName, + "accuracy", + Map[String, Any]( + ("df.name" -> accuracyMetricName), + ("miss" -> missColName), + ("total" -> totalColName), + ("matched" -> matchedColName), + ("tmst" -> GroupByColumn.tmst) + ), + resultPersistType(details, AccuracyInfo._Accuracy, MetricPersistType), + None + ) - missRecordsStep :: missStep :: totalStep :: accuracyMetricStep :: accuracyStep :: Nil + missRecordsStep :: missStep :: totalStep :: accuracyMetricStep :: accuracyStep :: Nil + } } case ProfilingType => { val sourceName = getNameOpt(details, ProfilingInfo._Source) match { @@ -325,29 +276,25 @@ case class GriffinDslAdaptor(dataSourceNames: Seq[String], val tailClause = analyzer.tailsExprs.map(_.desc).mkString(" ") - // 1. select statement -// val profilingSql = processType match { -// case BatchProcessType => { -// s"SELECT ${selClause} FROM ${sourceName} ${tailClause}" -// } -// case StreamingProcessType => { -// s"SELECT ${GroupByColumn.tmst}, ${selClause} FROM ${sourceName} ${tailClause}" + -// s" GROUP BY ${GroupByColumn.tmst}" -// } -// } - val profilingSql = { - s"SELECT `${GroupByColumn.tmst}`, ${selClause} FROM ${sourceName} ${tailClause} GROUP BY `${GroupByColumn.tmst}`" + if (!checkDataSourceExists(sourceName)) { + Nil + } else { + // 1. select statement + val profilingSql = { + s"SELECT `${GroupByColumn.tmst}`, ${selClause} FROM ${sourceName} ${tailClause} GROUP BY `${GroupByColumn.tmst}`" + } + val profilingMetricName = resultName(details, ProfilingInfo._Profiling) + val profilingStep = SparkSqlStep( + profilingMetricName, + profilingSql, + details, + resultPersistType(details, ProfilingInfo._Profiling, MetricPersistType), + None + ) + + profilingStep :: Nil } - val profilingMetricName = resultName(details, ProfilingInfo._Profiling) - val profilingStep = SparkSqlStep( - profilingMetricName, - profilingSql, - details, - resultPersistType(details, ProfilingInfo._Profiling, MetricPersistType), - None - ) - profilingStep :: Nil } case TimelinessType => { Nil @@ -356,4 +303,8 @@ case class GriffinDslAdaptor(dataSourceNames: Seq[String], } } + private def checkDataSourceExists(name: String): Boolean = { + RuleAdaptorGroup.dataChecker.existDataSourceName(name) + } + } diff --git a/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/RuleAdaptorGroup.scala b/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/RuleAdaptorGroup.scala index cbf065ab6..d0e6511f7 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/RuleAdaptorGroup.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/RuleAdaptorGroup.scala @@ -20,6 +20,7 @@ package org.apache.griffin.measure.rules.adaptor import org.apache.griffin.measure.algo.ProcessType import org.apache.griffin.measure.config.params.user._ +import org.apache.griffin.measure.process.check.DataChecker import org.apache.griffin.measure.rules.dsl._ import org.apache.griffin.measure.rules.step._ import org.apache.spark.sql.SQLContext @@ -33,10 +34,14 @@ object RuleAdaptorGroup { var dataSourceNames: Seq[String] = _ var functionNames: Seq[String] = _ + var dataChecker: DataChecker = _ + def init(sqlContext: SQLContext, dsNames: Seq[String]): Unit = { val functions = sqlContext.sql("show functions") functionNames = functions.map(_.getString(0)).collect dataSourceNames = dsNames + + dataChecker = DataChecker(sqlContext) } private def getDslType(param: Map[String, Any], defDslType: DslType) = { @@ -72,14 +77,18 @@ object RuleAdaptorGroup { // steps // } - def genConcreteRuleSteps(evaluateRuleParam: EvaluateRuleParam, adaptPhase: AdaptPhase): Seq[ConcreteRuleStep] = { + def genConcreteRuleSteps(evaluateRuleParam: EvaluateRuleParam, + adaptPhase: AdaptPhase + ): Seq[ConcreteRuleStep] = { val dslTypeStr = if (evaluateRuleParam.dslType == null) "" else evaluateRuleParam.dslType val defaultDslType = DslType(dslTypeStr) val ruleParams = evaluateRuleParam.rules genConcreteRuleSteps(ruleParams, defaultDslType, adaptPhase) } - def genConcreteRuleSteps(ruleParams: Seq[Map[String, Any]], defDslType: DslType, adaptPhase: AdaptPhase): Seq[ConcreteRuleStep] = { + def genConcreteRuleSteps(ruleParams: Seq[Map[String, Any]], + defDslType: DslType, adaptPhase: AdaptPhase + ): Seq[ConcreteRuleStep] = { val (steps, dsNames) = ruleParams.foldLeft((Seq[ConcreteRuleStep](), dataSourceNames)) { (res, param) => val (preSteps, preNames) = res val dslType = getDslType(param, defDslType) From 073d2b264170f29cb874f39641b7951845f2d628 Mon Sep 17 00:00:00 2001 From: Lionel Liu Date: Fri, 15 Sep 2017 13:27:03 +0800 Subject: [PATCH 074/111] streaming profiling --- .../measure/cache/info/TimeInfoCache.scala | 2 +- .../measure/data/source/DataSourceCache.scala | 6 +- .../process/engine/DataFrameOprEngine.scala | 42 +++++++++++- .../measure/process/engine/DqEngine.scala | 2 + .../measure/process/engine/DqEngines.scala | 12 +++- .../process/engine/SparkSqlEngine.scala | 28 +++++++- .../rules/adaptor/GriffinDslAdaptor.scala | 13 +++- .../measure/utils/HdfsFileDumpUtil.scala | 2 +- .../griffin/measure/utils/HdfsUtil.scala | 8 ++- .../config-test-profiling-streaming.json | 68 +++++++++++++++++++ .../measure/process/JsonParseTest.scala | 2 +- .../process/StreamingProcessTest.scala | 4 +- 12 files changed, 172 insertions(+), 17 deletions(-) create mode 100644 measure/src/test/resources/config-test-profiling-streaming.json diff --git a/measure/src/main/scala/org/apache/griffin/measure/cache/info/TimeInfoCache.scala b/measure/src/main/scala/org/apache/griffin/measure/cache/info/TimeInfoCache.scala index ac0acff7f..b581a584f 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/cache/info/TimeInfoCache.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/cache/info/TimeInfoCache.scala @@ -109,7 +109,7 @@ object TimeInfoCache extends Loggable with Serializable { case _ => -1 } } catch { - case _ => -1 + case e: Throwable => -1 } } diff --git a/measure/src/main/scala/org/apache/griffin/measure/data/source/DataSourceCache.scala b/measure/src/main/scala/org/apache/griffin/measure/data/source/DataSourceCache.scala index 0fb5205df..e2e1b8bb6 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/data/source/DataSourceCache.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/data/source/DataSourceCache.scala @@ -235,7 +235,7 @@ case class DataSourceCache(sqlContext: SQLContext, param: Map[String, Any], case head :: tail => { val (lb, ub) = head val curPaths = paths.flatMap { path => - val names = HdfsUtil.listSubPaths(path, "dir").toList + val names = HdfsUtil.listSubPathsByType(path, "dir").toList names.filter { name => str2Long(name) match { case Some(t) => (t >= lb) && (t <= ub) @@ -253,7 +253,7 @@ case class DataSourceCache(sqlContext: SQLContext, param: Map[String, Any], case Nil => paths case head :: tail => { val earlierPaths = paths.flatMap { path => - val names = HdfsUtil.listSubPaths(path, "dir").toList + val names = HdfsUtil.listSubPathsByType(path, "dir").toList names.filter { name => str2Long(name) match { case Some(t) => (t < head) @@ -262,7 +262,7 @@ case class DataSourceCache(sqlContext: SQLContext, param: Map[String, Any], }.map(HdfsUtil.getHdfsFilePath(path, _)) } val equalPaths = paths.flatMap { path => - val names = HdfsUtil.listSubPaths(path, "dir").toList + val names = HdfsUtil.listSubPathsByType(path, "dir").toList names.filter { name => str2Long(name) match { case Some(t) => (t == head) diff --git a/measure/src/main/scala/org/apache/griffin/measure/process/engine/DataFrameOprEngine.scala b/measure/src/main/scala/org/apache/griffin/measure/process/engine/DataFrameOprEngine.scala index 3bcc91984..111bfbff7 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/process/engine/DataFrameOprEngine.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/process/engine/DataFrameOprEngine.scala @@ -54,6 +54,10 @@ case class DataFrameOprEngine(sqlContext: SQLContext, @transient ssc: StreamingC val df = DataFrameOprs.accuracy(sqlContext, details) df.registerTempTable(name) } + case DataFrameOprs._clear => { + val df = DataFrameOprs.clear(sqlContext, details) + df.registerTempTable(name) + } case _ => { throw new Exception(s"df opr [ ${rule} ] not supported") } @@ -111,7 +115,31 @@ case class DataFrameOprEngine(sqlContext: SQLContext, @transient ssc: StreamingC }.toMap } catch { case e: Throwable => { - error(s"persist result ${name} error: ${e.getMessage}") + error(s"collect records ${name} error: ${e.getMessage}") + Map[Long, DataFrame]() + } + } + } + case _ => Map[Long, DataFrame]() + } + } + + def collectUpdateCacheDatas(ruleStep: ConcreteRuleStep, timeGroups: Iterable[Long]): Map[Long, DataFrame] = { + ruleStep match { + case DfOprStep(name, _, _, _, Some(ds)) => { + try { + val pdf = sqlContext.table(s"`${name}`") + timeGroups.flatMap { timeGroup => + try { + val tdf = pdf.filter(s"`${GroupByColumn.tmst}` = ${timeGroup}") + Some((timeGroup, tdf)) + } catch { + case e: Throwable => None + } + }.toMap + } catch { + case e: Throwable => { + error(s"collect update cache datas ${name} error: ${e.getMessage}") Map[Long, DataFrame]() } } @@ -180,7 +208,7 @@ case class DataFrameOprEngine(sqlContext: SQLContext, @transient ssc: StreamingC // } } catch { case e: Throwable => { - error(s"persist result ${name} error: ${e.getMessage}") + error(s"collect metrics ${name} error: ${e.getMessage}") // emptyMap Map[Long, Map[String, Any]]() } @@ -217,6 +245,7 @@ object DataFrameOprs { final val _fromJson = "from_json" final val _accuracy = "accuracy" + final val _clear = "clear" def fromJson(sqlContext: SQLContext, details: Map[String, Any]): DataFrame = { val _dfName = "df.name" @@ -291,6 +320,15 @@ object DataFrameOprs { } + def clear(sqlContext: SQLContext, details: Map[String, Any]): DataFrame = { + val _dfName = "df.name" + val dfName = details.getOrElse(_dfName, "").toString + + val df = sqlContext.table(s"`${dfName}`") + val emptyRdd = sqlContext.sparkContext.emptyRDD[Row] + sqlContext.createDataFrame(emptyRdd, df.schema) + } + } diff --git a/measure/src/main/scala/org/apache/griffin/measure/process/engine/DqEngine.scala b/measure/src/main/scala/org/apache/griffin/measure/process/engine/DqEngine.scala index 3c63756dc..c53bc939b 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/process/engine/DqEngine.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/process/engine/DqEngine.scala @@ -40,4 +40,6 @@ trait DqEngine extends Loggable with Serializable { def collectMetrics(ruleStep: ConcreteRuleStep): Map[Long, Map[String, Any]] def collectRecords(ruleStep: ConcreteRuleStep, timeGroups: Iterable[Long]): Map[Long, DataFrame] + + def collectUpdateCacheDatas(ruleStep: ConcreteRuleStep, timeGroups: Iterable[Long]): Map[Long, DataFrame] } diff --git a/measure/src/main/scala/org/apache/griffin/measure/process/engine/DqEngines.scala b/measure/src/main/scala/org/apache/griffin/measure/process/engine/DqEngines.scala index 4113ca24a..361599a97 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/process/engine/DqEngines.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/process/engine/DqEngines.scala @@ -91,9 +91,8 @@ case class DqEngines(engines: Seq[DqEngine]) extends DqEngine { def updateDataSources(ruleSteps: Seq[ConcreteRuleStep], dataSources: Seq[DataSource], timeGroups: Iterable[Long]): Unit = { - val recordSteps = ruleSteps.filter(_.persistType == RecordPersistType) + val recordSteps = ruleSteps.filter(_.updateDataSource.nonEmpty) recordSteps.foreach { step => - val name = step.name val udpateDataSources = dataSources.filter { ds => step.updateDataSource match { case Some(dsName) if (dsName == ds.name) => true @@ -101,7 +100,7 @@ case class DqEngines(engines: Seq[DqEngine]) extends DqEngine { } } if (udpateDataSources.size > 0) { - val records = collectRecords(step, timeGroups) + val records = collectUpdateCacheDatas(step, timeGroups) records.foreach { pair => val (t, recs) = pair @@ -178,6 +177,13 @@ case class DqEngines(engines: Seq[DqEngine]) extends DqEngine { // if (ret.isEmpty) warn(s"collect records warn: no records collected for ${ruleStep}") ret } + def collectUpdateCacheDatas(ruleStep: ConcreteRuleStep, timeGroups: Iterable[Long]): Map[Long, DataFrame] = { + val ret = engines.foldLeft(Map[Long, DataFrame]()) { (ret, engine) => + ret ++ engine.collectUpdateCacheDatas(ruleStep, timeGroups) + } + // if (ret.isEmpty) warn(s"collect records warn: no records collected for ${ruleStep}") + ret + } def collectMetrics(ruleStep: ConcreteRuleStep): Map[Long, Map[String, Any]] = { val ret = engines.foldLeft(Map[Long, Map[String, Any]]()) { (ret, engine) => ret ++ engine.collectMetrics(ruleStep) diff --git a/measure/src/main/scala/org/apache/griffin/measure/process/engine/SparkSqlEngine.scala b/measure/src/main/scala/org/apache/griffin/measure/process/engine/SparkSqlEngine.scala index 0b16013ad..4398754ac 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/process/engine/SparkSqlEngine.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/process/engine/SparkSqlEngine.scala @@ -100,7 +100,31 @@ case class SparkSqlEngine(sqlContext: SQLContext, @transient ssc: StreamingConte }.toMap } catch { case e: Throwable => { - error(s"persist result ${name} error: ${e.getMessage}") + error(s"collect records ${name} error: ${e.getMessage}") + Map[Long, DataFrame]() + } + } + } + case _ => Map[Long, DataFrame]() + } + } + + def collectUpdateCacheDatas(ruleStep: ConcreteRuleStep, timeGroups: Iterable[Long]): Map[Long, DataFrame] = { + ruleStep match { + case SparkSqlStep(name, _, _, _, Some(ds)) => { + try { + val pdf = sqlContext.table(s"`${name}`") + timeGroups.flatMap { timeGroup => + try { + val tdf = pdf.filter(s"`${GroupByColumn.tmst}` = ${timeGroup}") + Some((timeGroup, tdf)) + } catch { + case e: Throwable => None + } + }.toMap + } catch { + case e: Throwable => { + error(s"collect update cache datas ${name} error: ${e.getMessage}") Map[Long, DataFrame]() } } @@ -169,7 +193,7 @@ case class SparkSqlEngine(sqlContext: SQLContext, @transient ssc: StreamingConte // } } catch { case e: Throwable => { - error(s"persist result ${name} error: ${e.getMessage}") + error(s"collect metrics ${name} error: ${e.getMessage}") // emptyMap Map[Long, Map[String, Any]]() } diff --git a/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/GriffinDslAdaptor.scala b/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/GriffinDslAdaptor.scala index b2e9729f8..767fd98b5 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/GriffinDslAdaptor.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/GriffinDslAdaptor.scala @@ -292,7 +292,18 @@ case class GriffinDslAdaptor(dataSourceNames: Seq[String], None ) - profilingStep :: Nil + // 2. clear processed data + val clearDataSourceStep = DfOprStep( + s"${profilingMetricName}_clear", + "clear", + Map[String, Any]( + ("df.name" -> sourceName) + ), + NonePersistType, + Some(sourceName) + ) + + profilingStep :: clearDataSourceStep :: Nil } } diff --git a/measure/src/main/scala/org/apache/griffin/measure/utils/HdfsFileDumpUtil.scala b/measure/src/main/scala/org/apache/griffin/measure/utils/HdfsFileDumpUtil.scala index 8a608ff0b..416f56738 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/utils/HdfsFileDumpUtil.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/utils/HdfsFileDumpUtil.scala @@ -68,7 +68,7 @@ object HdfsFileDumpUtil { def remove(path: String, filename: String, withSuffix: Boolean): Unit = { if (withSuffix) { - val files = HdfsUtil.listSubPaths(path, "file") + val files = HdfsUtil.listSubPathsByType(path, "file") val patternFiles = files.filter(samePattern(_, filename)) patternFiles.foreach { f => val rmPath = HdfsUtil.getHdfsFilePath(path, f) diff --git a/measure/src/main/scala/org/apache/griffin/measure/utils/HdfsUtil.scala b/measure/src/main/scala/org/apache/griffin/measure/utils/HdfsUtil.scala index 079359275..6f513c168 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/utils/HdfsUtil.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/utils/HdfsUtil.scala @@ -96,7 +96,7 @@ object HdfsUtil { // } // } - def listSubPaths(dirPath: String, subType: String, fullPath: Boolean = false): Iterable[String] = { + def listSubPathsByType(dirPath: String, subType: String, fullPath: Boolean = false): Iterable[String] = { val path = new Path(dirPath) try { val fileStatusArray = dfs.listStatus(path) @@ -117,4 +117,10 @@ object HdfsUtil { } } } + + def listSubPathsByTypes(dirPath: String, subTypes: Iterable[String], fullPath: Boolean = false): Iterable[String] = { + subTypes.flatMap { subType => + listSubPathsByType(dirPath, subType, fullPath) + } + } } diff --git a/measure/src/test/resources/config-test-profiling-streaming.json b/measure/src/test/resources/config-test-profiling-streaming.json new file mode 100644 index 000000000..7282ecabd --- /dev/null +++ b/measure/src/test/resources/config-test-profiling-streaming.json @@ -0,0 +1,68 @@ +{ + "name": "prof_streaming", + + "process.type": "streaming", + + "data.sources": [ + { + "name": "source", + "connectors": [ + { + "type": "kafka", + "version": "0.8", + "config": { + "kafka.config": { + "bootstrap.servers": "10.149.247.156:9092", + "group.id": "group1", + "auto.offset.reset": "smallest", + "auto.commit.enable": "false" + }, + "topics": "sss", + "key.type": "java.lang.String", + "value.type": "java.lang.String" + }, + "pre.proc": [ + { + "dsl.type": "df-opr", + "name": "${s1}", + "rule": "from_json", + "persist.type": "cache", + "details": { + "df.name": "${this}" + } + }, + { + "dsl.type": "spark-sql", + "name": "${this}", + "rule": "select name, age from ${s1}" + } + ] + } + ], + "cache": { + "file.path": "hdfs://localhost/griffin/streaming/dump/source", + "info.path": "source", + "ready.time.interval": "10s", + "ready.time.delay": "0", + "time.range": ["-1m", "0"] + } + } + ], + + "evaluateRule": { + "rules": [ + { + "dsl.type": "griffin-dsl", + "dq.type": "profiling", + "rule": "source.name.count()", + "details": { + "source": "source", + "profiling": { + "name": "count", + "persist.type": "metric" + } + } + } + ] + } +} \ No newline at end of file diff --git a/measure/src/test/scala/org/apache/griffin/measure/process/JsonParseTest.scala b/measure/src/test/scala/org/apache/griffin/measure/process/JsonParseTest.scala index 04683453a..0d72ae1ee 100644 --- a/measure/src/test/scala/org/apache/griffin/measure/process/JsonParseTest.scala +++ b/measure/src/test/scala/org/apache/griffin/measure/process/JsonParseTest.scala @@ -508,7 +508,7 @@ class JsonParseTest extends FunSuite with Matchers with BeforeAndAfter with Logg case head :: tail => { val (lb, ub) = head val curPaths = paths.flatMap { path => - val names = HdfsUtil.listSubPaths(path, "dir").toList + val names = HdfsUtil.listSubPathsByType(path, "dir").toList println(names) names.filter { name => str2Long(name) match { diff --git a/measure/src/test/scala/org/apache/griffin/measure/process/StreamingProcessTest.scala b/measure/src/test/scala/org/apache/griffin/measure/process/StreamingProcessTest.scala index 5e4424f17..51902bc79 100644 --- a/measure/src/test/scala/org/apache/griffin/measure/process/StreamingProcessTest.scala +++ b/measure/src/test/scala/org/apache/griffin/measure/process/StreamingProcessTest.scala @@ -36,8 +36,8 @@ import scala.util.{Failure, Success, Try} class StreamingProcessTest extends FunSuite with Matchers with BeforeAndAfter with Loggable { val envFile = "src/test/resources/env-streaming.json" - val confFile = "src/test/resources/config-test-accuracy-streaming.json" -// val confFile = "src/test/resources/config-test-accuracy.json" +// val confFile = "src/test/resources/config-test-accuracy-streaming.json" + val confFile = "src/test/resources/config-test-profiling-streaming.json" val envFsType = "local" val userFsType = "local" From e427591427f14a7f226936708ade6bc5bb3804b4 Mon Sep 17 00:00:00 2001 From: Lionel Liu Date: Fri, 15 Sep 2017 13:51:55 +0800 Subject: [PATCH 075/111] refactor dq engine inherit way --- .../measure/process/BatchDqProcess.scala | 2 +- .../measure/process/StreamingDqProcess.scala | 2 +- .../process/engine/DataFrameOprEngine.scala | 172 +-------------- .../measure/process/engine/DqEngine.scala | 6 - .../process/engine/DqEngineFactory.scala | 10 +- .../process/engine/SparkDqEngine.scala | 130 +++++++++++ .../process/engine/SparkSqlEngine.scala | 205 +----------------- .../measure/process/BatchProcessTest.scala | 4 +- .../process/StreamingProcessTest.scala | 4 +- 9 files changed, 143 insertions(+), 392 deletions(-) create mode 100644 measure/src/main/scala/org/apache/griffin/measure/process/engine/SparkDqEngine.scala diff --git a/measure/src/main/scala/org/apache/griffin/measure/process/BatchDqProcess.scala b/measure/src/main/scala/org/apache/griffin/measure/process/BatchDqProcess.scala index 4b4678ba1..c3b796620 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/process/BatchDqProcess.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/process/BatchDqProcess.scala @@ -77,7 +77,7 @@ case class BatchDqProcess(allParam: AllParam) extends DqProcess { persist.start(applicationId) // get dq engines - val dqEngines = DqEngineFactory.genDqEngines(sqlContext, null) + val dqEngines = DqEngineFactory.genDqEngines(sqlContext) // generate data sources val dataSources = DataSourceFactory.genDataSources(sqlContext, null, dqEngines, userParam.dataSources, metricName) diff --git a/measure/src/main/scala/org/apache/griffin/measure/process/StreamingDqProcess.scala b/measure/src/main/scala/org/apache/griffin/measure/process/StreamingDqProcess.scala index ddf76bdae..2acd3c61e 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/process/StreamingDqProcess.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/process/StreamingDqProcess.scala @@ -94,7 +94,7 @@ case class StreamingDqProcess(allParam: AllParam) extends DqProcess { persist.start(applicationId) // get dq engines - val dqEngines = DqEngineFactory.genDqEngines(sqlContext, ssc) + val dqEngines = DqEngineFactory.genDqEngines(sqlContext) // generate data sources val dataSources = DataSourceFactory.genDataSources(sqlContext, ssc, dqEngines, userParam.dataSources, metricName) diff --git a/measure/src/main/scala/org/apache/griffin/measure/process/engine/DataFrameOprEngine.scala b/measure/src/main/scala/org/apache/griffin/measure/process/engine/DataFrameOprEngine.scala index 111bfbff7..dd685f227 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/process/engine/DataFrameOprEngine.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/process/engine/DataFrameOprEngine.scala @@ -34,12 +34,7 @@ import org.apache.spark.sql.types.{LongType, StringType, StructField, StructType import org.apache.spark.sql.{DataFrame, Row, SQLContext} import org.apache.spark.streaming.StreamingContext -case class DataFrameOprEngine(sqlContext: SQLContext, @transient ssc: StreamingContext - ) extends DqEngine { - -// def genDataSource(dataSourceParam: DataSourceParam): Option[DirectDataSource] = { -// DataSourceFactory.genDataSource(sqlContext, ssc, dataSourceParam) -// } +case class DataFrameOprEngine(sqlContext: SQLContext) extends SparkDqEngine { def runRuleStep(ruleStep: ConcreteRuleStep): Boolean = { ruleStep match { @@ -74,171 +69,6 @@ case class DataFrameOprEngine(sqlContext: SQLContext, @transient ssc: StreamingC } } -// def persistRecords(ruleStep: ConcreteRuleStep, timeGroups: Iterable[Long], persistFactory: PersistFactory): Boolean = { -// val curTime = new Date().getTime -// ruleStep match { -// case DfOprStep(name, _, _, RecordPersistType) => { -// try { -// val pdf = sqlContext.table(s"`${name}`") -// val records = pdf.toJSON -// -// persist.persistRecords(records, name) -// -// val recordLog = s"[ ${name} ] persist records" -// persist.log(curTime, recordLog) -// -// true -// } catch { -// case e: Throwable => { -// error(s"persist result ${name} error: ${e.getMessage}") -// false -// } -// } -// } -// case _ => false -// } -// true -// } - - def collectRecords(ruleStep: ConcreteRuleStep, timeGroups: Iterable[Long]): Map[Long, DataFrame] = { - ruleStep match { - case DfOprStep(name, _, _, RecordPersistType, _) => { - try { - val pdf = sqlContext.table(s"`${name}`") - timeGroups.flatMap { timeGroup => - try { - val tdf = pdf.filter(s"`${GroupByColumn.tmst}` = ${timeGroup}") - Some((timeGroup, tdf)) - } catch { - case e: Throwable => None - } - }.toMap - } catch { - case e: Throwable => { - error(s"collect records ${name} error: ${e.getMessage}") - Map[Long, DataFrame]() - } - } - } - case _ => Map[Long, DataFrame]() - } - } - - def collectUpdateCacheDatas(ruleStep: ConcreteRuleStep, timeGroups: Iterable[Long]): Map[Long, DataFrame] = { - ruleStep match { - case DfOprStep(name, _, _, _, Some(ds)) => { - try { - val pdf = sqlContext.table(s"`${name}`") - timeGroups.flatMap { timeGroup => - try { - val tdf = pdf.filter(s"`${GroupByColumn.tmst}` = ${timeGroup}") - Some((timeGroup, tdf)) - } catch { - case e: Throwable => None - } - }.toMap - } catch { - case e: Throwable => { - error(s"collect update cache datas ${name} error: ${e.getMessage}") - Map[Long, DataFrame]() - } - } - } - case _ => Map[Long, DataFrame]() - } - } - - def collectMetrics(ruleStep: ConcreteRuleStep): Map[Long, Map[String, Any]] = { - val emptyMap = Map[String, Any]() - ruleStep match { - case DfOprStep(name, _, _, MetricPersistType, _) => { - try { - val pdf = sqlContext.table(s"`${name}`") - val records = pdf.toJSON.collect() - - val pairs = records.flatMap { rec => - try { - val value = JsonUtil.toAnyMap(rec) - value.get(GroupByColumn.tmst) match { - case Some(t) => { - val key = t.toString.toLong - Some((key, value)) - } - case _ => None - } - } catch { - case e: Throwable => None - } - } - val groupedPairs = pairs.foldLeft(Map[Long, Seq[Map[String, Any]]]()) { (ret, pair) => - val (k, v) = pair - ret.get(k) match { - case Some(seq) => ret + (k -> (seq :+ v)) - case _ => ret + (k -> (v :: Nil)) - } - } - groupedPairs.mapValues { vs => - if (vs.size > 1) { - Map[String, Any]((name -> vs)) - } else { - vs.headOption.getOrElse(emptyMap) - } - } - -// if (ruleStep.isGroupMetric) { -// val arr = records.flatMap { rec => -// try { -// Some(JsonUtil.toAnyMap(rec)) -// } catch { -// case e: Throwable => None -// } -// } -// Map[String, Any]((name -> arr)) -// } else { -// records.headOption match { -// case Some(head) => { -// try { -// JsonUtil.toAnyMap(head) -// } catch { -// case e: Throwable => emptyMap -// } -// } -// case _ => emptyMap -// } -// } - } catch { - case e: Throwable => { - error(s"collect metrics ${name} error: ${e.getMessage}") -// emptyMap - Map[Long, Map[String, Any]]() - } - } - } -// case _ => emptyMap - case _ => Map[Long, Map[String, Any]]() - } - } - -// def persistResults(ruleStep: ConcreteRuleStep, persist: Persist): Boolean = { -// val curTime = new Date().getTime -// ruleStep match { -// case DfOprStep(name, _, _) => { -// try { -// val nonLog = s"[ ${name} ] not persisted" -// persist.log(curTime, nonLog) -// -// true -// } catch { -// case e: Throwable => { -// error(s"persist result ${ruleStep.name} error: ${e.getMessage}") -// false -// } -// } -// } -// case _ => false -// } -// } - } object DataFrameOprs { diff --git a/measure/src/main/scala/org/apache/griffin/measure/process/engine/DqEngine.scala b/measure/src/main/scala/org/apache/griffin/measure/process/engine/DqEngine.scala index c53bc939b..383d48a38 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/process/engine/DqEngine.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/process/engine/DqEngine.scala @@ -29,14 +29,8 @@ import org.apache.spark.sql.DataFrame trait DqEngine extends Loggable with Serializable { -// def genDataSource(dataSourceParam: DataSourceParam): Option[DirectDataSource] - def runRuleStep(ruleStep: ConcreteRuleStep): Boolean -// def persistResults(ruleSteps: Seq[ConcreteRuleStep], persist: Persist, persistType: PersistType): Boolean - -// def persistRecords(ruleStep: ConcreteRuleStep, timeGroups: Iterable[Long], persistFactory: PersistFactory): Boolean - def collectMetrics(ruleStep: ConcreteRuleStep): Map[Long, Map[String, Any]] def collectRecords(ruleStep: ConcreteRuleStep, timeGroups: Iterable[Long]): Map[Long, DataFrame] diff --git a/measure/src/main/scala/org/apache/griffin/measure/process/engine/DqEngineFactory.scala b/measure/src/main/scala/org/apache/griffin/measure/process/engine/DqEngineFactory.scala index 033c53e3e..e075584df 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/process/engine/DqEngineFactory.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/process/engine/DqEngineFactory.scala @@ -29,17 +29,17 @@ object DqEngineFactory { private final val SparkSqlEngineType = "spark-sql" private final val DataFrameOprEngineType = "df-opr" - def genDqEngines(sqlContext: SQLContext, ssc: StreamingContext): DqEngines = { + def genDqEngines(sqlContext: SQLContext): DqEngines = { val engines = engineTypes.flatMap { et => - genDqEngine(et, sqlContext, ssc) + genDqEngine(et, sqlContext) } DqEngines(engines) } - private def genDqEngine(engineType: String, sqlContext: SQLContext, ssc: StreamingContext): Option[DqEngine] = { + private def genDqEngine(engineType: String, sqlContext: SQLContext): Option[DqEngine] = { engineType match { - case SparkSqlEngineType => Some(SparkSqlEngine(sqlContext, ssc)) - case DataFrameOprEngineType => Some(DataFrameOprEngine(sqlContext, ssc)) + case SparkSqlEngineType => Some(SparkSqlEngine(sqlContext)) + case DataFrameOprEngineType => Some(DataFrameOprEngine(sqlContext)) case _ => None } } diff --git a/measure/src/main/scala/org/apache/griffin/measure/process/engine/SparkDqEngine.scala b/measure/src/main/scala/org/apache/griffin/measure/process/engine/SparkDqEngine.scala new file mode 100644 index 000000000..a4b1b6c43 --- /dev/null +++ b/measure/src/main/scala/org/apache/griffin/measure/process/engine/SparkDqEngine.scala @@ -0,0 +1,130 @@ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ +package org.apache.griffin.measure.process.engine + +import org.apache.griffin.measure.data.connector.GroupByColumn +import org.apache.griffin.measure.log.Loggable +import org.apache.griffin.measure.rules.dsl.{MetricPersistType, RecordPersistType} +import org.apache.griffin.measure.rules.step._ +import org.apache.griffin.measure.utils.JsonUtil +import org.apache.spark.sql.{DataFrame, SQLContext} + +trait SparkDqEngine extends DqEngine { + + val sqlContext: SQLContext + + def collectMetrics(ruleStep: ConcreteRuleStep): Map[Long, Map[String, Any]] = { + val emptyMap = Map[String, Any]() + ruleStep match { + case step: ConcreteRuleStep if (step.persistType == MetricPersistType) => { + val name = step.name + try { + val pdf = sqlContext.table(s"`${name}`") + val records = pdf.toJSON.collect() + + val pairs = records.flatMap { rec => + try { + val value = JsonUtil.toAnyMap(rec) + value.get(GroupByColumn.tmst) match { + case Some(t) => { + val key = t.toString.toLong + Some((key, value)) + } + case _ => None + } + } catch { + case e: Throwable => None + } + } + val groupedPairs = pairs.foldLeft(Map[Long, Seq[Map[String, Any]]]()) { (ret, pair) => + val (k, v) = pair + ret.get(k) match { + case Some(seq) => ret + (k -> (seq :+ v)) + case _ => ret + (k -> (v :: Nil)) + } + } + groupedPairs.mapValues { vs => + if (vs.size > 1) { + Map[String, Any]((name -> vs)) + } else { + vs.headOption.getOrElse(emptyMap) + } + } + } catch { + case e: Throwable => { + error(s"collect metrics ${name} error: ${e.getMessage}") + Map[Long, Map[String, Any]]() + } + } + } + case _ => Map[Long, Map[String, Any]]() + } + } + + def collectRecords(ruleStep: ConcreteRuleStep, timeGroups: Iterable[Long]): Map[Long, DataFrame] = { + ruleStep match { + case step: ConcreteRuleStep if (step.persistType == RecordPersistType) => { + val name = step.name + try { + val pdf = sqlContext.table(s"`${name}`") + timeGroups.flatMap { timeGroup => + try { + val tdf = pdf.filter(s"`${GroupByColumn.tmst}` = ${timeGroup}") + Some((timeGroup, tdf)) + } catch { + case e: Throwable => None + } + }.toMap + } catch { + case e: Throwable => { + error(s"collect records ${name} error: ${e.getMessage}") + Map[Long, DataFrame]() + } + } + } + case _ => Map[Long, DataFrame]() + } + } + + def collectUpdateCacheDatas(ruleStep: ConcreteRuleStep, timeGroups: Iterable[Long]): Map[Long, DataFrame] = { + ruleStep match { + case step: ConcreteRuleStep if (step.updateDataSource.nonEmpty) => { + val name = step.name + try { + val pdf = sqlContext.table(s"`${name}`") + timeGroups.flatMap { timeGroup => + try { + val tdf = pdf.filter(s"`${GroupByColumn.tmst}` = ${timeGroup}") + Some((timeGroup, tdf)) + } catch { + case e: Throwable => None + } + }.toMap + } catch { + case e: Throwable => { + error(s"collect update cache datas ${name} error: ${e.getMessage}") + Map[Long, DataFrame]() + } + } + } + case _ => Map[Long, DataFrame]() + } + } + +} diff --git a/measure/src/main/scala/org/apache/griffin/measure/process/engine/SparkSqlEngine.scala b/measure/src/main/scala/org/apache/griffin/measure/process/engine/SparkSqlEngine.scala index 4398754ac..ef745cb44 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/process/engine/SparkSqlEngine.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/process/engine/SparkSqlEngine.scala @@ -31,12 +31,7 @@ import org.apache.spark.rdd.RDD import org.apache.spark.sql.{DataFrame, GroupedData, SQLContext} import org.apache.spark.streaming.StreamingContext -case class SparkSqlEngine(sqlContext: SQLContext, @transient ssc: StreamingContext - ) extends DqEngine { - -// def genDataSource(dataSourceParam: DataSourceParam): Option[DirectDataSource] = { -// DataSourceFactory.genDataSource(sqlContext, ssc, dataSourceParam) -// } +case class SparkSqlEngine(sqlContext: SQLContext) extends SparkDqEngine { def runRuleStep(ruleStep: ConcreteRuleStep): Boolean = { ruleStep match { @@ -56,204 +51,6 @@ case class SparkSqlEngine(sqlContext: SQLContext, @transient ssc: StreamingConte } } -// def persistRecords(ruleStep: ConcreteRuleStep, timeGroups: Iterable[Long], persistFactory: PersistFactory): Boolean = { -// val curTime = new Date().getTime -// ruleStep match { -// case SparkSqlStep(name, _, _, RecordPersistType) => { -// try { -// val pdf = sqlContext.table(s"`${name}`") -// val records = pdf.toJSON -// -// timeGroups.foreach { timeGroup => -// val persist = persistFactory.getPersists(timeGroup) -// -// persist.persistRecords(records, name) -// -//// val recordLog = s"[ ${name} ] persist records" -//// persist.log(curTime, recordLog) -// } -// -// true -// } catch { -// case e: Throwable => { -// error(s"persist result ${name} error: ${e.getMessage}") -// false -// } -// } -// } -// case _ => false -// } -// } - - def collectRecords(ruleStep: ConcreteRuleStep, timeGroups: Iterable[Long]): Map[Long, DataFrame] = { - ruleStep match { - case SparkSqlStep(name, _, _, RecordPersistType, _) => { - try { - val pdf = sqlContext.table(s"`${name}`") - timeGroups.flatMap { timeGroup => - try { - val tdf = pdf.filter(s"`${GroupByColumn.tmst}` = ${timeGroup}") - Some((timeGroup, tdf)) - } catch { - case e: Throwable => None - } - }.toMap - } catch { - case e: Throwable => { - error(s"collect records ${name} error: ${e.getMessage}") - Map[Long, DataFrame]() - } - } - } - case _ => Map[Long, DataFrame]() - } - } - - def collectUpdateCacheDatas(ruleStep: ConcreteRuleStep, timeGroups: Iterable[Long]): Map[Long, DataFrame] = { - ruleStep match { - case SparkSqlStep(name, _, _, _, Some(ds)) => { - try { - val pdf = sqlContext.table(s"`${name}`") - timeGroups.flatMap { timeGroup => - try { - val tdf = pdf.filter(s"`${GroupByColumn.tmst}` = ${timeGroup}") - Some((timeGroup, tdf)) - } catch { - case e: Throwable => None - } - }.toMap - } catch { - case e: Throwable => { - error(s"collect update cache datas ${name} error: ${e.getMessage}") - Map[Long, DataFrame]() - } - } - } - case _ => Map[Long, DataFrame]() - } - } - - def collectMetrics(ruleStep: ConcreteRuleStep): Map[Long, Map[String, Any]] = { - val emptyMap = Map[String, Any]() - ruleStep match { - case SparkSqlStep(name, _, _, MetricPersistType, _) => { - try { - val pdf = sqlContext.table(s"`${name}`") - val records = pdf.toJSON.collect() - - val pairs = records.flatMap { rec => - try { - val value = JsonUtil.toAnyMap(rec) - value.get(GroupByColumn.tmst) match { - case Some(t) => { - val key = t.toString.toLong - Some((key, value)) - } - case _ => None - } - } catch { - case e: Throwable => None - } - } - val groupedPairs = pairs.foldLeft(Map[Long, Seq[Map[String, Any]]]()) { (ret, pair) => - val (k, v) = pair - ret.get(k) match { - case Some(seq) => ret + (k -> (seq :+ v)) - case _ => ret + (k -> (v :: Nil)) - } - } - groupedPairs.mapValues { vs => - if (vs.size > 1) { - Map[String, Any]((name -> vs)) - } else { - vs.headOption.getOrElse(emptyMap) - } - } - -// if (ruleStep.isGroupMetric) { -// val arr = records.flatMap { rec => -// try { -// Some(JsonUtil.toAnyMap(rec)) -// } catch { -// case e: Throwable => None -// } -// } -// Map[String, Any]((name -> arr)) -// } else { -// records.headOption match { -// case Some(head) => { -// try { -// JsonUtil.toAnyMap(head) -// } catch { -// case e: Throwable => emptyMap -// } -// } -// case _ => emptyMap -// } -// } - } catch { - case e: Throwable => { - error(s"collect metrics ${name} error: ${e.getMessage}") -// emptyMap - Map[Long, Map[String, Any]]() - } - } - } -// case _ => emptyMap - case _ => Map[Long, Map[String, Any]]() - } - } - -// def persistResults(ruleSteps: Seq[ConcreteRuleStep], persist: Persist, persistType: PersistType): Boolean = { -// val curTime = new Date().getTime -// persistType match { -// case RecordPersistType => { -// ; -// } -// } -// -// -// ruleStep match { -// case SparkSqlStep(name, _, persistType) => { -// try { -// persistType match { -// case RecordPersistType => { -// val pdf = sqlContext.table(s"`${name}`") -// val records = pdf.toJSON -// -// persist.persistRecords(records, name) -// -// val recordLog = s"[ ${name} ] persist records" -// persist.log(curTime, recordLog) -// } -// case MetricPersistType => { -// val pdf = sqlContext.table(s"`${name}`") -// val recordRdd = pdf.toJSON -// -// val metrics = recordRdd.collect -// persist.persistMetrics(metrics, name) -// -// val metricLog = s"[ ${name} ] persist metric \n${metrics.mkString("\n")}" -// persist.log(curTime, metricLog) -// } -// case _ => { -// val nonLog = s"[ ${name} ] not persisted" -// persist.log(curTime, nonLog) -// } -// } -// -// true -// } catch { -// case e: Throwable => { -// error(s"persist result ${name} error: ${e.getMessage}") -// false -// } -// } -// } -// case _ => false -// } -// } - } diff --git a/measure/src/test/scala/org/apache/griffin/measure/process/BatchProcessTest.scala b/measure/src/test/scala/org/apache/griffin/measure/process/BatchProcessTest.scala index 3a2b3e84a..41e61c5ad 100644 --- a/measure/src/test/scala/org/apache/griffin/measure/process/BatchProcessTest.scala +++ b/measure/src/test/scala/org/apache/griffin/measure/process/BatchProcessTest.scala @@ -36,8 +36,8 @@ import scala.util.{Failure, Success, Try} class BatchProcessTest extends FunSuite with Matchers with BeforeAndAfter with Loggable { val envFile = "src/test/resources/env-test.json" -// val confFile = "src/test/resources/config-test-profiling.json" - val confFile = "src/test/resources/config-test-accuracy.json" + val confFile = "src/test/resources/config-test-profiling.json" +// val confFile = "src/test/resources/config-test-accuracy.json" val envFsType = "local" val userFsType = "local" diff --git a/measure/src/test/scala/org/apache/griffin/measure/process/StreamingProcessTest.scala b/measure/src/test/scala/org/apache/griffin/measure/process/StreamingProcessTest.scala index 51902bc79..489aa3325 100644 --- a/measure/src/test/scala/org/apache/griffin/measure/process/StreamingProcessTest.scala +++ b/measure/src/test/scala/org/apache/griffin/measure/process/StreamingProcessTest.scala @@ -36,8 +36,8 @@ import scala.util.{Failure, Success, Try} class StreamingProcessTest extends FunSuite with Matchers with BeforeAndAfter with Loggable { val envFile = "src/test/resources/env-streaming.json" -// val confFile = "src/test/resources/config-test-accuracy-streaming.json" - val confFile = "src/test/resources/config-test-profiling-streaming.json" + val confFile = "src/test/resources/config-test-accuracy-streaming.json" +// val confFile = "src/test/resources/config-test-profiling-streaming.json" val envFsType = "local" val userFsType = "local" From 7c5125c5f1143d9128e324ae218a18cc17424ebc Mon Sep 17 00:00:00 2001 From: Lionel Liu Date: Mon, 18 Sep 2017 14:43:37 +0800 Subject: [PATCH 076/111] vi --- measure/src/test/resources/config-test-profiling-streaming.json | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/measure/src/test/resources/config-test-profiling-streaming.json b/measure/src/test/resources/config-test-profiling-streaming.json index 7282ecabd..8a5e31f26 100644 --- a/measure/src/test/resources/config-test-profiling-streaming.json +++ b/measure/src/test/resources/config-test-profiling-streaming.json @@ -54,7 +54,7 @@ { "dsl.type": "griffin-dsl", "dq.type": "profiling", - "rule": "source.name.count()", + "rule": "source.name.count() where source.age > 5", "details": { "source": "source", "profiling": { From 1a78d23a60572e542c9f8cbb7f43cbc670529e1d Mon Sep 17 00:00:00 2001 From: Lionel Liu Date: Mon, 18 Sep 2017 18:10:28 +0800 Subject: [PATCH 077/111] param map get value --- .../batch/AvroBatchDataConnector.scala | 5 +- .../batch/HiveBatchDataConnector.scala | 7 +- .../batch/TextDirBatchDataConnector.scala | 107 ++++++++++++ .../KafkaStreamingDataConnector.scala | 8 +- .../KafkaStreamingStringDataConnector.scala | 2 +- .../measure/data/source/DataSourceCache.scala | 9 +- .../griffin/measure/persist/HdfsPersist.scala | 5 +- .../griffin/measure/persist/HttpPersist.scala | 5 +- .../measure/persist/LoggerPersist.scala | 3 +- .../rules/adaptor/GriffinDslAdaptor.scala | 5 +- .../griffin/measure/utils/ParamUtil.scala | 164 ++++++++++++++++++ ...onfig-test-accuracy-streaming-multids.json | 143 +++++++++++++++ .../measure/process/BatchProcessTest.scala | 4 +- .../griffin/measure/utils/ParamUtilTest.scala | 50 ++++++ 14 files changed, 493 insertions(+), 24 deletions(-) create mode 100644 measure/src/main/scala/org/apache/griffin/measure/data/connector/batch/TextDirBatchDataConnector.scala create mode 100644 measure/src/main/scala/org/apache/griffin/measure/utils/ParamUtil.scala create mode 100644 measure/src/test/resources/config-test-accuracy-streaming-multids.json create mode 100644 measure/src/test/scala/org/apache/griffin/measure/utils/ParamUtilTest.scala diff --git a/measure/src/main/scala/org/apache/griffin/measure/data/connector/batch/AvroBatchDataConnector.scala b/measure/src/main/scala/org/apache/griffin/measure/data/connector/batch/AvroBatchDataConnector.scala index d9bef1227..3d0fcfdfd 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/data/connector/batch/AvroBatchDataConnector.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/data/connector/batch/AvroBatchDataConnector.scala @@ -26,6 +26,7 @@ import org.apache.griffin.measure.rule.{ExprValueUtil, RuleExprs} import org.apache.griffin.measure.utils.HdfsUtil import org.apache.spark.rdd.RDD import org.apache.spark.sql.{DataFrame, SQLContext} +import org.apache.griffin.measure.utils.ParamUtil._ import scala.util.Try @@ -38,8 +39,8 @@ case class AvroBatchDataConnector(sqlContext: SQLContext, dqEngines: DqEngines, val FilePath = "file.path" val FileName = "file.name" - val filePath = config.getOrElse(FilePath, "").toString - val fileName = config.getOrElse(FileName, "").toString + val filePath = config.getString(FilePath, "") + val fileName = config.getString(FileName, "") val concreteFileFullPath = if (pathPrefix) s"${filePath}${fileName}" else fileName diff --git a/measure/src/main/scala/org/apache/griffin/measure/data/connector/batch/HiveBatchDataConnector.scala b/measure/src/main/scala/org/apache/griffin/measure/data/connector/batch/HiveBatchDataConnector.scala index 652d2b243..87fa4f94a 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/data/connector/batch/HiveBatchDataConnector.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/data/connector/batch/HiveBatchDataConnector.scala @@ -28,6 +28,7 @@ import org.apache.spark.sql.hive.HiveContext import org.apache.spark.sql.{DataFrame, SQLContext} import scala.util.{Success, Try} +import org.apache.griffin.measure.utils.ParamUtil._ // data connector for hive case class HiveBatchDataConnector(sqlContext: SQLContext, dqEngines: DqEngines, dcParam: DataConnectorParam @@ -43,9 +44,9 @@ case class HiveBatchDataConnector(sqlContext: SQLContext, dqEngines: DqEngines, val TableName = "table.name" val Partitions = "partitions" - val database = config.getOrElse(Database, "default").toString - val tableName = config.getOrElse(TableName, "").toString - val partitionsString = config.getOrElse(Partitions, "").toString + val database = config.getString(Database, "default") + val tableName = config.getString(TableName, "") + val partitionsString = config.getString(Partitions, "") val concreteTableName = s"${database}.${tableName}" val partitions = partitionsString.split(";").map(s => s.split(",").map(_.trim)) diff --git a/measure/src/main/scala/org/apache/griffin/measure/data/connector/batch/TextDirBatchDataConnector.scala b/measure/src/main/scala/org/apache/griffin/measure/data/connector/batch/TextDirBatchDataConnector.scala new file mode 100644 index 000000000..127c88bba --- /dev/null +++ b/measure/src/main/scala/org/apache/griffin/measure/data/connector/batch/TextDirBatchDataConnector.scala @@ -0,0 +1,107 @@ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ +package org.apache.griffin.measure.data.connector.batch + +import org.apache.griffin.measure.config.params.user.DataConnectorParam +import org.apache.griffin.measure.process.engine.DqEngines +import org.apache.griffin.measure.utils.HdfsUtil +import org.apache.spark.sql.{DataFrame, SQLContext} +import org.apache.griffin.measure.utils.ParamUtil._ + +// data connector for avro file +case class TextDirBatchDataConnector(sqlContext: SQLContext, dqEngines: DqEngines, dcParam: DataConnectorParam + ) extends BatchDataConnector { + + val config = dcParam.config + + val DirPath = "dir.path" + val Recursive = "recursive" + val Removable = "removable" + val FilePrefix = "file.prefix" + + val dirPath = config.getString(DirPath, "") + val recursive = config.getBoolean(Recursive, true) + val removable = config.getBoolean(Removable, false) + val filePrefix = config.getString(FilePrefix, "_") + + private def dirExist(): Boolean = { + HdfsUtil.existPath(dirPath) + } + + def data(ms: Long): Option[DataFrame] = { + // fixme + None +// try { +// val df = sqlContext.read.format("com.databricks.spark.avro").load(concreteFileFullPath) +// val dfOpt = Some(df) +// val preDfOpt = preProcess(dfOpt, ms) +// preDfOpt +// } catch { +// case e: Throwable => { +// error(s"load avro file ${concreteFileFullPath} fails") +// None +// } +// } + } + +// def available(): Boolean = { +// (!concreteFileFullPath.isEmpty) && fileExist +// } + +// def init(): Unit = {} + +// def metaData(): Try[Iterable[(String, String)]] = { +// Try { +// val st = sqlContext.read.format("com.databricks.spark.avro").load(concreteFileFullPath).schema +// st.fields.map(f => (f.name, f.dataType.typeName)) +// } +// } + +// def data(): Try[RDD[(Product, (Map[String, Any], Map[String, Any]))]] = { +// Try { +// loadDataFile.flatMap { row => +// // generate cache data +// val cacheExprValueMaps = ExprValueUtil.genExprValueMaps(Some(row), ruleExprs.cacheExprs, constFinalExprValueMap) +// val finalExprValueMaps = ExprValueUtil.updateExprValueMaps(ruleExprs.finalCacheExprs, cacheExprValueMaps) +// +// // data info +// val dataInfoMap: Map[String, Any] = DataInfo.cacheInfoList.map { info => +// try { +// (info.key -> row.getAs[info.T](info.key)) +// } catch { +// case e: Throwable => info.defWrap +// } +// }.toMap +// +// finalExprValueMaps.flatMap { finalExprValueMap => +// val groupbyData: Seq[AnyRef] = ruleExprs.groupbyExprs.flatMap { expr => +// expr.calculate(finalExprValueMap) match { +// case Some(v) => Some(v.asInstanceOf[AnyRef]) +// case _ => None +// } +// } +// val key = toTuple(groupbyData) +// +// Some((key, (finalExprValueMap, dataInfoMap))) +// } +// } +// } +// } + +} diff --git a/measure/src/main/scala/org/apache/griffin/measure/data/connector/streaming/KafkaStreamingDataConnector.scala b/measure/src/main/scala/org/apache/griffin/measure/data/connector/streaming/KafkaStreamingDataConnector.scala index 964425dab..41de2175a 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/data/connector/streaming/KafkaStreamingDataConnector.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/data/connector/streaming/KafkaStreamingDataConnector.scala @@ -22,6 +22,7 @@ import kafka.serializer.Decoder import org.apache.spark.streaming.dstream.InputDStream import scala.util.{Failure, Success, Try} +import org.apache.griffin.measure.utils.ParamUtil._ trait KafkaStreamingDataConnector extends StreamingDataConnector { @@ -33,11 +34,8 @@ trait KafkaStreamingDataConnector extends StreamingDataConnector { val KafkaConfig = "kafka.config" val Topics = "topics" - val kafkaConfig = config.get(KafkaConfig) match { - case Some(map: Map[String, Any]) => map.mapValues(_.toString).map(identity) - case _ => Map[String, String]() - } - val topics = config.getOrElse(Topics, "").toString + val kafkaConfig = config.getAnyRef(KafkaConfig, Map[String, String]()) + val topics = config.getString(Topics, "") def available(): Boolean = { true diff --git a/measure/src/main/scala/org/apache/griffin/measure/data/connector/streaming/KafkaStreamingStringDataConnector.scala b/measure/src/main/scala/org/apache/griffin/measure/data/connector/streaming/KafkaStreamingStringDataConnector.scala index e78734ede..5e0413e76 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/data/connector/streaming/KafkaStreamingStringDataConnector.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/data/connector/streaming/KafkaStreamingStringDataConnector.scala @@ -41,7 +41,7 @@ case class KafkaStreamingStringDataConnector(sqlContext: SQLContext, val valueColName = "value" val schema = StructType(Array( - StructField("value", StringType) + StructField(valueColName, StringType) )) def createDStream(topicSet: Set[String]): InputDStream[(K, V)] = { diff --git a/measure/src/main/scala/org/apache/griffin/measure/data/source/DataSourceCache.scala b/measure/src/main/scala/org/apache/griffin/measure/data/source/DataSourceCache.scala index e2e1b8bb6..64877ad97 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/data/source/DataSourceCache.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/data/source/DataSourceCache.scala @@ -29,6 +29,7 @@ import org.apache.spark.rdd.RDD import org.apache.spark.sql.{DataFrame, SQLContext} import scala.util.{Failure, Success} +import org.apache.griffin.measure.utils.ParamUtil._ case class DataSourceCache(sqlContext: SQLContext, param: Map[String, Any], metricName: String, index: Int @@ -45,10 +46,10 @@ case class DataSourceCache(sqlContext: SQLContext, param: Map[String, Any], val defFilePath = s"hdfs:///griffin/cache/${metricName}/${index}" val defInfoPath = s"${index}" - val filePath: String = param.getOrElse(_FilePath, defFilePath).toString - val cacheInfoPath: String = param.getOrElse(_InfoPath, defInfoPath).toString - val readyTimeInterval: Long = TimeUtil.milliseconds(param.getOrElse(_ReadyTimeInterval, "1m").toString).getOrElse(60000L) - val readyTimeDelay: Long = TimeUtil.milliseconds(param.getOrElse(_ReadyTimeDelay, "1m").toString).getOrElse(60000L) + val filePath: String = param.getString(_FilePath, defFilePath) + val cacheInfoPath: String = param.getString(_InfoPath, defInfoPath) + val readyTimeInterval: Long = TimeUtil.milliseconds(param.getString(_ReadyTimeInterval, "1m")).getOrElse(60000L) + val readyTimeDelay: Long = TimeUtil.milliseconds(param.getString(_ReadyTimeDelay, "1m")).getOrElse(60000L) val deltaTimeRange: (Long, Long) = { def negative(n: Long): Long = if (n <= 0) n else 0 param.get(_TimeRange) match { diff --git a/measure/src/main/scala/org/apache/griffin/measure/persist/HdfsPersist.scala b/measure/src/main/scala/org/apache/griffin/measure/persist/HdfsPersist.scala index 200ef8424..97cebc92e 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/persist/HdfsPersist.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/persist/HdfsPersist.scala @@ -26,6 +26,7 @@ import org.apache.spark.rdd.RDD import org.apache.spark.sql.DataFrame import scala.util.Try +import org.apache.griffin.measure.utils.ParamUtil._ // persist result and data to hdfs case class HdfsPersist(config: Map[String, Any], metricName: String, timeStamp: Long) extends Persist { @@ -35,8 +36,8 @@ case class HdfsPersist(config: Map[String, Any], metricName: String, timeStamp: val MaxLinesPerFile = "max.lines.per.file" val path = config.getOrElse(Path, "").toString - val maxPersistLines = try { config.getOrElse(MaxPersistLines, -1).toString.toInt } catch { case _: Throwable => -1 } - val maxLinesPerFile = try { config.getOrElse(MaxLinesPerFile, 10000).toString.toLong } catch { case _: Throwable => 10000 } + val maxPersistLines = config.getInt(MaxPersistLines, -1) + val maxLinesPerFile = config.getLong(MaxLinesPerFile, 10000) val separator = "/" diff --git a/measure/src/main/scala/org/apache/griffin/measure/persist/HttpPersist.scala b/measure/src/main/scala/org/apache/griffin/measure/persist/HttpPersist.scala index 6d5e9a06b..3a4fa67a1 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/persist/HttpPersist.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/persist/HttpPersist.scala @@ -24,6 +24,7 @@ import org.apache.spark.rdd.RDD import org.apache.spark.sql.DataFrame import scala.util.Try +import org.apache.griffin.measure.utils.ParamUtil._ // persist result by http way case class HttpPersist(config: Map[String, Any], metricName: String, timeStamp: Long) extends Persist { @@ -31,8 +32,8 @@ case class HttpPersist(config: Map[String, Any], metricName: String, timeStamp: val Api = "api" val Method = "method" - val api = config.getOrElse(Api, "").toString - val method = config.getOrElse(Method, "post").toString + val api = config.getString(Api, "") + val method = config.getString(Method, "post") val _Value = "value" diff --git a/measure/src/main/scala/org/apache/griffin/measure/persist/LoggerPersist.scala b/measure/src/main/scala/org/apache/griffin/measure/persist/LoggerPersist.scala index 5b2be3e7c..12672822b 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/persist/LoggerPersist.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/persist/LoggerPersist.scala @@ -24,13 +24,14 @@ import org.apache.griffin.measure.result._ import org.apache.griffin.measure.utils.{HdfsUtil, JsonUtil} import org.apache.spark.rdd.RDD import org.apache.spark.sql.DataFrame +import org.apache.griffin.measure.utils.ParamUtil._ // persist result and data to hdfs case class LoggerPersist(config: Map[String, Any], metricName: String, timeStamp: Long) extends Persist { val MaxLogLines = "max.log.lines" - val maxLogLines = try { config.getOrElse(MaxLogLines, 100).toString.toInt } catch { case _: Throwable => 100 } + val maxLogLines = config.getInt(MaxLogLines, 100) def available(): Boolean = true diff --git a/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/GriffinDslAdaptor.scala b/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/GriffinDslAdaptor.scala index 767fd98b5..f721a13b3 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/GriffinDslAdaptor.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/GriffinDslAdaptor.scala @@ -25,6 +25,7 @@ import org.apache.griffin.measure.rules.dsl.analyzer._ import org.apache.griffin.measure.rules.dsl.expr._ import org.apache.griffin.measure.rules.dsl.parser.GriffinDslParser import org.apache.griffin.measure.rules.step._ +import org.apache.griffin.measure.utils.ParamUtil._ case class GriffinDslAdaptor(dataSourceNames: Seq[String], functionNames: Seq[String], @@ -36,7 +37,7 @@ case class GriffinDslAdaptor(dataSourceNames: Seq[String], val _PersistType = "persist.type" val _UpdateDataSource = "update.data.source" def getNameOpt(param: Map[String, Any]): Option[String] = param.get(_Name).map(_.toString) - def getPersistType(param: Map[String, Any]): PersistType = PersistType(param.getOrElse(_PersistType, "").toString) + def getPersistType(param: Map[String, Any]): PersistType = PersistType(param.getString(_PersistType, "")) def getUpdateDataSourceOpt(param: Map[String, Any]): Option[String] = param.get(_UpdateDataSource).map(_.toString) } object AccuracyInfo { @@ -76,7 +77,7 @@ case class GriffinDslAdaptor(dataSourceNames: Seq[String], val _dqType = "dq.type" - protected def getDqType(param: Map[String, Any]) = DqType(param.getOrElse(_dqType, "").toString) + protected def getDqType(param: Map[String, Any]) = DqType(param.getString(_dqType, "")) val filteredFunctionNames = functionNames.filter { fn => fn.matches("""^[a-zA-Z_]\w*$""") diff --git a/measure/src/main/scala/org/apache/griffin/measure/utils/ParamUtil.scala b/measure/src/main/scala/org/apache/griffin/measure/utils/ParamUtil.scala new file mode 100644 index 000000000..7954b6d09 --- /dev/null +++ b/measure/src/main/scala/org/apache/griffin/measure/utils/ParamUtil.scala @@ -0,0 +1,164 @@ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ +package org.apache.griffin.measure.utils + +object ParamUtil { + + implicit class ParamMap(params: Map[String, Any]) { + def getAny(key: String, defValue: Any): Any = { + params.get(key) match { + case Some(v) => v + case _ => defValue + } + } + + def getAnyRef[T](key: String, defValue: T)(implicit m: Manifest[T]): T = { + params.get(key) match { + case Some(v: T) => v + case _ => defValue + } + } + + def getString(key: String, defValue: String): String = { + try { + params.get(key) match { + case Some(v: String) => v.toString + case Some(v) => v.toString + case _ => defValue + } + } catch { + case _: Throwable => defValue + } + } + + def getByte(key: String, defValue: Byte): Byte = { + try { + params.get(key) match { + case Some(v: String) => v.toByte + case Some(v: Byte) => v.toByte + case Some(v: Short) => v.toByte + case Some(v: Int) => v.toByte + case Some(v: Long) => v.toByte + case Some(v: Float) => v.toByte + case Some(v: Double) => v.toByte + case _ => defValue + } + } catch { + case _: Throwable => defValue + } + } + + def getShort(key: String, defValue: Short): Short = { + try { + params.get(key) match { + case Some(v: String) => v.toShort + case Some(v: Byte) => v.toShort + case Some(v: Short) => v.toShort + case Some(v: Int) => v.toShort + case Some(v: Long) => v.toShort + case Some(v: Float) => v.toShort + case Some(v: Double) => v.toShort + case _ => defValue + } + } catch { + case _: Throwable => defValue + } + } + + def getInt(key: String, defValue: Int): Int = { + try { + params.get(key) match { + case Some(v: String) => v.toInt + case Some(v: Byte) => v.toInt + case Some(v: Short) => v.toInt + case Some(v: Int) => v.toInt + case Some(v: Long) => v.toInt + case Some(v: Float) => v.toInt + case Some(v: Double) => v.toInt + case _ => defValue + } + } catch { + case _: Throwable => defValue + } + } + + def getLong(key: String, defValue: Long): Long = { + try { + params.get(key) match { + case Some(v: String) => v.toLong + case Some(v: Byte) => v.toLong + case Some(v: Short) => v.toLong + case Some(v: Int) => v.toLong + case Some(v: Long) => v.toLong + case Some(v: Float) => v.toLong + case Some(v: Double) => v.toLong + case _ => defValue + } + } catch { + case _: Throwable => defValue + } + } + + def getFloat(key: String, defValue: Float): Float = { + try { + params.get(key) match { + case Some(v: String) => v.toFloat + case Some(v: Byte) => v.toFloat + case Some(v: Short) => v.toFloat + case Some(v: Int) => v.toFloat + case Some(v: Long) => v.toFloat + case Some(v: Float) => v.toFloat + case Some(v: Double) => v.toFloat + case _ => defValue + } + } catch { + case _: Throwable => defValue + } + } + + def getDouble(key: String, defValue: Double): Double = { + try { + params.get(key) match { + case Some(v: String) => v.toDouble + case Some(v: Byte) => v.toDouble + case Some(v: Short) => v.toDouble + case Some(v: Int) => v.toDouble + case Some(v: Long) => v.toDouble + case Some(v: Float) => v.toDouble + case Some(v: Double) => v.toDouble + case _ => defValue + } + } catch { + case _: Throwable => defValue + } + } + + def getBoolean(key: String, defValue: Boolean): Boolean = { + try { + params.get(key) match { + case Some(v: String) => v.toBoolean + case _ => defValue + } + } catch { + case _: Throwable => defValue + } + } + } + +} diff --git a/measure/src/test/resources/config-test-accuracy-streaming-multids.json b/measure/src/test/resources/config-test-accuracy-streaming-multids.json new file mode 100644 index 000000000..45e40fba7 --- /dev/null +++ b/measure/src/test/resources/config-test-accuracy-streaming-multids.json @@ -0,0 +1,143 @@ +{ + "name": "accu_streaming", + + "process.type": "streaming", + + "data.sources": [ + { + "name": "source", + "connectors": [ + { + "type": "kafka", + "version": "0.8", + "config": { + "kafka.config": { + "bootstrap.servers": "10.149.247.156:9092", + "group.id": "group1", + "auto.offset.reset": "smallest", + "auto.commit.enable": "false" + }, + "topics": "sss", + "key.type": "java.lang.String", + "value.type": "java.lang.String" + }, + "pre.proc": [ + { + "dsl.type": "df-opr", + "name": "${s1}", + "rule": "from_json", + "persist.type": "cache", + "details": { + "df.name": "${this}" + } + }, + { + "dsl.type": "spark-sql", + "name": "${this}", + "rule": "select name, age from ${s1}" + } + ] + }, + { + "type": "text-dir", + "config": { + "dir.path": "hdfs://localhost/griffin/text", + "recursive": true, + "removable": false + }, + "pre.proc": [ + { + "dsl.type": "df-opr", + "name": "${s1}", + "rule": "from_json", + "persist.type": "cache", + "details": { + "df.name": "${this}" + } + }, + { + "dsl.type": "spark-sql", + "name": "${this}", + "rule": "select name, age from ${s1}" + } + ] + } + ], + "cache": { + "file.path": "hdfs://localhost/griffin/streaming/dump/source", + "info.path": "source", + "ready.time.interval": "10s", + "ready.time.delay": "0", + "time.range": ["-2m", "0"] + } + }, { + "name": "target", + "connectors": [ + { + "type": "kafka", + "version": "0.8", + "config": { + "kafka.config": { + "bootstrap.servers": "10.149.247.156:9092", + "group.id": "group1", + "auto.offset.reset": "smallest", + "auto.commit.enable": "false" + }, + "topics": "ttt", + "key.type": "java.lang.String", + "value.type": "java.lang.String" + }, + "pre.proc": [ + { + "dsl.type": "df-opr", + "name": "${t1}", + "rule": "from_json", + "persist.type": "cache", + "details": { + "df.name": "${this}" + } + }, + { + "dsl.type": "spark-sql", + "name": "${this}", + "rule": "select name, age from ${t1}" + } + ] + } + ], + "cache": { + "file.path": "hdfs://localhost/griffin/streaming/dump/target", + "info.path": "target", + "ready.time.interval": "10s", + "ready.time.delay": "0", + "time.range": ["-2m", "0"] + } + } + ], + + "evaluateRule": { + "rules": [ + { + "dsl.type": "griffin-dsl", + "dq.type": "accuracy", + "rule": "source.name = target.name and source.age = target.age", + "details": { + "source": "source", + "target": "target", + "miss.records": { + "name": "miss.records", + "persist.type": "record", + "update.data.source": "source" + }, + "accuracy": { + "name": "accu", + "persist.type": "metric" + }, + "miss": "miss_count", + "total": "total_count", + "matched": "matched_count" + } + } + ] + } +} \ No newline at end of file diff --git a/measure/src/test/scala/org/apache/griffin/measure/process/BatchProcessTest.scala b/measure/src/test/scala/org/apache/griffin/measure/process/BatchProcessTest.scala index 41e61c5ad..3a2b3e84a 100644 --- a/measure/src/test/scala/org/apache/griffin/measure/process/BatchProcessTest.scala +++ b/measure/src/test/scala/org/apache/griffin/measure/process/BatchProcessTest.scala @@ -36,8 +36,8 @@ import scala.util.{Failure, Success, Try} class BatchProcessTest extends FunSuite with Matchers with BeforeAndAfter with Loggable { val envFile = "src/test/resources/env-test.json" - val confFile = "src/test/resources/config-test-profiling.json" -// val confFile = "src/test/resources/config-test-accuracy.json" +// val confFile = "src/test/resources/config-test-profiling.json" + val confFile = "src/test/resources/config-test-accuracy.json" val envFsType = "local" val userFsType = "local" diff --git a/measure/src/test/scala/org/apache/griffin/measure/utils/ParamUtilTest.scala b/measure/src/test/scala/org/apache/griffin/measure/utils/ParamUtilTest.scala new file mode 100644 index 000000000..5a54b1159 --- /dev/null +++ b/measure/src/test/scala/org/apache/griffin/measure/utils/ParamUtilTest.scala @@ -0,0 +1,50 @@ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ +package org.apache.griffin.measure.utils + +import java.io.{BufferedReader, InputStreamReader} + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.{FileSystem, Path} +import org.apache.spark.sql.hive.HiveContext +import org.apache.spark.{SparkConf, SparkContext} +import org.junit.runner.RunWith +import org.scalatest.junit.JUnitRunner +import org.scalatest.{BeforeAndAfter, FunSuite, Matchers} + +@RunWith(classOf[JUnitRunner]) +class ParamUtilTest extends FunSuite with Matchers with BeforeAndAfter { + + test ("test param util") { + val params = Map[String, Any]( + ("a" -> "321"), + ("b" -> 123), + ("c" -> 3.2), + ("d" -> (213 :: 321 :: Nil)) + ) + + import ParamUtil._ + + params.getString("a", "") should be ("321") + params.getInt("b", 0) should be (123) + params.getBoolean("c", false) should be (false) + params.getAnyRef("d", List[Int]()) should be ((213 :: 321 :: Nil)) + } + +} From 230a1ed173c1cc68183e8bdd86db90f045860231 Mon Sep 17 00:00:00 2001 From: Lionel Liu Date: Wed, 20 Sep 2017 18:21:29 +0800 Subject: [PATCH 078/111] multi data source --- .../data/connector/DataConnectorFactory.scala | 2 + .../batch/TextDirBatchDataConnector.scala | 67 +++++++++++++------ .../measure/data/source/DataSource.scala | 13 +++- .../griffin/measure/utils/HdfsUtil.scala | 65 ++++++++++++------ ...onfig-test-accuracy-streaming-multids.json | 5 +- .../data/connector/ConnectorTest.scala | 5 +- .../process/StreamingProcessTest.scala | 3 +- 7 files changed, 113 insertions(+), 47 deletions(-) diff --git a/measure/src/main/scala/org/apache/griffin/measure/data/connector/DataConnectorFactory.scala b/measure/src/main/scala/org/apache/griffin/measure/data/connector/DataConnectorFactory.scala index f8cde73ba..9d190bbd9 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/data/connector/DataConnectorFactory.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/data/connector/DataConnectorFactory.scala @@ -42,6 +42,7 @@ object DataConnectorFactory { val HiveRegex = """^(?i)hive$""".r val AvroRegex = """^(?i)avro$""".r + val TextDirRegex = """^(?i)text-dir$""".r val KafkaRegex = """^(?i)kafka$""".r @@ -59,6 +60,7 @@ object DataConnectorFactory { conType match { case HiveRegex() => HiveBatchDataConnector(sqlContext, dqEngines, dataConnectorParam) case AvroRegex() => AvroBatchDataConnector(sqlContext, dqEngines, dataConnectorParam) + case TextDirRegex() => TextDirBatchDataConnector(sqlContext, dqEngines, dataConnectorParam) case KafkaRegex() => { // val ksdcTry = getStreamingDataConnector(ssc, dataConnectorParam) // val cdcTry = getCacheDataConnector(sqlContext, dataConnectorParam.cache) diff --git a/measure/src/main/scala/org/apache/griffin/measure/data/connector/batch/TextDirBatchDataConnector.scala b/measure/src/main/scala/org/apache/griffin/measure/data/connector/batch/TextDirBatchDataConnector.scala index 127c88bba..abc547bc3 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/data/connector/batch/TextDirBatchDataConnector.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/data/connector/batch/TextDirBatchDataConnector.scala @@ -31,33 +31,62 @@ case class TextDirBatchDataConnector(sqlContext: SQLContext, dqEngines: DqEngine val config = dcParam.config val DirPath = "dir.path" - val Recursive = "recursive" - val Removable = "removable" - val FilePrefix = "file.prefix" + val DataDirDepth = "data.dir.depth" + val SuccessFile = "success.file" + val DoneFile = "done.file" val dirPath = config.getString(DirPath, "") - val recursive = config.getBoolean(Recursive, true) - val removable = config.getBoolean(Removable, false) - val filePrefix = config.getString(FilePrefix, "_") + val dataDirDepth = config.getInt(DataDirDepth, 0) + val successFile = config.getString(SuccessFile, "_SUCCESS") + val doneFile = config.getString(DoneFile, "_DONE") + + val ignoreFilePrefix = "_" private def dirExist(): Boolean = { HdfsUtil.existPath(dirPath) } def data(ms: Long): Option[DataFrame] = { - // fixme - None -// try { -// val df = sqlContext.read.format("com.databricks.spark.avro").load(concreteFileFullPath) -// val dfOpt = Some(df) -// val preDfOpt = preProcess(dfOpt, ms) -// preDfOpt -// } catch { -// case e: Throwable => { -// error(s"load avro file ${concreteFileFullPath} fails") -// None -// } -// } + try { + val dataDirs = listSubDirs(dirPath :: Nil, dataDirDepth, readable) + // touch done file for read dirs + dataDirs.foreach(dir => touchDone(dir)) + + val validDataDirs = dataDirs.filter(dir => !emptyDir(dir)) + + if (validDataDirs.size > 0) { + val df = sqlContext.read.text(validDataDirs: _*) + val dfOpt = Some(df) + val preDfOpt = preProcess(dfOpt, ms) + preDfOpt + } else { + None + } + } catch { + case e: Throwable => { + error(s"load text dir ${dirPath} fails: ${e.getMessage}") + None + } + } + } + + private def listSubDirs(paths: Seq[String], depth: Int, filteFunc: (String) => Boolean): Seq[String] = { + val subDirs = paths.flatMap { path => HdfsUtil.listSubPathsByType(path, "dir", true) } + if (depth <= 0) { + subDirs.filter(filteFunc) + } else { + listSubDirs(subDirs, depth, filteFunc) + } + } + + private def readable(dir: String): Boolean = isSuccess(dir) && !isDone(dir) + private def isDone(dir: String): Boolean = HdfsUtil.existFileInDir(dir, doneFile) + private def isSuccess(dir: String): Boolean = HdfsUtil.existFileInDir(dir, successFile) + + private def touchDone(dir: String): Unit = HdfsUtil.createEmptyFile(HdfsUtil.getHdfsFilePath(dir, doneFile)) + + private def emptyDir(dir: String): Boolean = { + HdfsUtil.listSubPathsByType(dir, "file").filter(!_.startsWith(ignoreFilePrefix)).size == 0 } // def available(): Boolean = { diff --git a/measure/src/main/scala/org/apache/griffin/measure/data/source/DataSource.scala b/measure/src/main/scala/org/apache/griffin/measure/data/source/DataSource.scala index 93dc84b9a..8e0f73143 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/data/source/DataSource.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/data/source/DataSource.scala @@ -23,7 +23,7 @@ import org.apache.griffin.measure.data.connector.batch._ import org.apache.griffin.measure.data.connector.streaming._ import org.apache.griffin.measure.log.Loggable import org.apache.spark.rdd.RDD -import org.apache.spark.sql.{DataFrame, SQLContext} +import org.apache.spark.sql.{DataFrame, Row, SQLContext} case class DataSource(sqlContext: SQLContext, name: String, @@ -79,7 +79,16 @@ case class DataSource(sqlContext: SQLContext, private def unionDataFrames(df1: DataFrame, df2: DataFrame): DataFrame = { try { - df1 unionAll df2 + val cols = df1.columns + val rdd2 = df2.map{ row => + val values = cols.map { col => + row.getAs[Any](col) + } + Row(values: _*) + } + val ndf2 = sqlContext.createDataFrame(rdd2, df1.schema) + df1 unionAll ndf2 +// df1 unionAll df2 } catch { case e: Throwable => df1 } diff --git a/measure/src/main/scala/org/apache/griffin/measure/utils/HdfsUtil.scala b/measure/src/main/scala/org/apache/griffin/measure/utils/HdfsUtil.scala index 6f513c168..69f63beed 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/utils/HdfsUtil.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/utils/HdfsUtil.scala @@ -18,10 +18,11 @@ under the License. */ package org.apache.griffin.measure.utils +import org.apache.griffin.measure.log.Loggable import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FSDataInputStream, FSDataOutputStream, FileSystem, Path} -object HdfsUtil { +object HdfsUtil extends Loggable { private val seprator = "/" @@ -32,8 +33,17 @@ object HdfsUtil { private val dfs = FileSystem.get(conf) def existPath(filePath: String): Boolean = { - val path = new Path(filePath) - dfs.exists(path) + try { + val path = new Path(filePath) + dfs.exists(path) + } catch { + case e: Throwable => false + } + } + + def existFileInDir(dirPath: String, fileName: String): Boolean = { + val filePath = getHdfsFilePath(dirPath, fileName) + existPath(filePath) } def createFile(filePath: String): FSDataOutputStream = { @@ -75,8 +85,12 @@ object HdfsUtil { } def deleteHdfsPath(dirPath: String): Unit = { - val path = new Path(dirPath) - if (dfs.exists(path)) dfs.delete(path, true) + try { + val path = new Path(dirPath) + if (dfs.exists(path)) dfs.delete(path, true) + } catch { + case e: Throwable => error(s"delete path [${dirPath}] error: ${e.getMessage}") + } } // def listPathFiles(dirPath: String): Iterable[String] = { @@ -97,25 +111,27 @@ object HdfsUtil { // } def listSubPathsByType(dirPath: String, subType: String, fullPath: Boolean = false): Iterable[String] = { - val path = new Path(dirPath) - try { - val fileStatusArray = dfs.listStatus(path) - fileStatusArray.filter { fileStatus => - subType match { - case "dir" => fileStatus.isDirectory - case "file" => fileStatus.isFile - case _ => true + if (existPath(dirPath)) { + try { + val path = new Path(dirPath) + val fileStatusArray = dfs.listStatus(path) + fileStatusArray.filter { fileStatus => + subType match { + case "dir" => fileStatus.isDirectory + case "file" => fileStatus.isFile + case _ => true + } + }.map { fileStatus => + val fname = fileStatus.getPath.getName + if (fullPath) getHdfsFilePath(dirPath, fname) else fname + } + } catch { + case e: Throwable => { + warn(s"list path [${dirPath}] warn: ${e.getMessage}") + Nil } - }.map { fileStatus => - val fname = fileStatus.getPath.getName - if (fullPath) getHdfsFilePath(dirPath, fname) else fname - } - } catch { - case e: Throwable => { - println(s"list path files error: ${e.getMessage}") - Nil } - } + } else Nil } def listSubPathsByTypes(dirPath: String, subTypes: Iterable[String], fullPath: Boolean = false): Iterable[String] = { @@ -123,4 +139,9 @@ object HdfsUtil { listSubPathsByType(dirPath, subType, fullPath) } } + + def fileNameFromPath(filePath: String): String = { + val path = new Path(filePath) + path.getName + } } diff --git a/measure/src/test/resources/config-test-accuracy-streaming-multids.json b/measure/src/test/resources/config-test-accuracy-streaming-multids.json index 45e40fba7..18532b0ad 100644 --- a/measure/src/test/resources/config-test-accuracy-streaming-multids.json +++ b/measure/src/test/resources/config-test-accuracy-streaming-multids.json @@ -42,8 +42,9 @@ "type": "text-dir", "config": { "dir.path": "hdfs://localhost/griffin/text", - "recursive": true, - "removable": false + "data.dir.depth": 0, + "success.file": "_SUCCESS", + "done.file": "_DONE" }, "pre.proc": [ { diff --git a/measure/src/test/scala/org/apache/griffin/measure/data/connector/ConnectorTest.scala b/measure/src/test/scala/org/apache/griffin/measure/data/connector/ConnectorTest.scala index 0e7c9f8be..978863d43 100644 --- a/measure/src/test/scala/org/apache/griffin/measure/data/connector/ConnectorTest.scala +++ b/measure/src/test/scala/org/apache/griffin/measure/data/connector/ConnectorTest.scala @@ -25,14 +25,17 @@ import kafka.serializer.StringDecoder import org.apache.griffin.measure.algo.streaming.TimingProcess import org.apache.griffin.measure.cache.info.InfoCacheInstance import org.apache.griffin.measure.config.params.env._ -import org.apache.griffin.measure.config.params.user.{DataCacheParam, DataConnectorParam, EvaluateRuleParam} +import org.apache.griffin.measure.config.params.user.{DataConnectorParam, EvaluateRuleParam} import org.apache.griffin.measure.config.reader.ParamRawStringReader +import org.apache.griffin.measure.data.connector.batch.TextDirBatchDataConnector +import org.apache.griffin.measure.process.engine.DqEngines import org.apache.griffin.measure.result.{DataInfo, TimeStampInfo} import org.apache.griffin.measure.rule.expr.{Expr, StatementExpr} import org.apache.griffin.measure.rule._ import org.apache.griffin.measure.utils.{HdfsFileDumpUtil, HdfsUtil, TimeUtil} import org.apache.griffin.measure.rule.{DataTypeCalculationUtil, ExprValueUtil, RuleExprs} import org.apache.spark.rdd.RDD +import org.apache.spark.sql.hive.HiveContext import org.apache.spark.sql.types.{DataType, StructField, StructType} import org.apache.spark.sql.{DataFrame, Row, SQLContext} import org.apache.spark.streaming.dstream.InputDStream diff --git a/measure/src/test/scala/org/apache/griffin/measure/process/StreamingProcessTest.scala b/measure/src/test/scala/org/apache/griffin/measure/process/StreamingProcessTest.scala index 489aa3325..9e6cdf854 100644 --- a/measure/src/test/scala/org/apache/griffin/measure/process/StreamingProcessTest.scala +++ b/measure/src/test/scala/org/apache/griffin/measure/process/StreamingProcessTest.scala @@ -36,7 +36,8 @@ import scala.util.{Failure, Success, Try} class StreamingProcessTest extends FunSuite with Matchers with BeforeAndAfter with Loggable { val envFile = "src/test/resources/env-streaming.json" - val confFile = "src/test/resources/config-test-accuracy-streaming.json" + val confFile = "src/test/resources/config-test-accuracy-streaming-multids.json" +// val confFile = "src/test/resources/config-test-accuracy-streaming.json" // val confFile = "src/test/resources/config-test-profiling-streaming.json" val envFsType = "local" From 6015e5e684c7db20f8bfb2bb4b56dd8f7344c7cf Mon Sep 17 00:00:00 2001 From: Lionel Liu Date: Fri, 22 Sep 2017 14:39:59 +0800 Subject: [PATCH 079/111] refactor --- .../griffin/measure/algo/AccuracyAlgo.scala | 24 -- .../apache/griffin/measure/algo/Algo.scala | 34 -- .../griffin/measure/algo/MeasureType.scala | 26 -- .../griffin/measure/algo/ProfileAlgo.scala | 23 -- .../algo/batch/BatchAccuracyAlgo.scala | 190 --------- .../measure/algo/batch/BatchProfileAlgo.scala | 162 -------- .../measure/algo/core/AccuracyCore.scala | 103 ----- .../measure/algo/core/ProfileCore.scala | 73 ---- .../streaming/StreamingAccuracyAlgo.scala | 369 ------------------ .../streaming/StreamingAccuracyProcess.scala | 234 ----------- .../config/params/user/RuleParam.scala | 34 -- .../data/connector/DataConnector.scala | 6 +- .../Algo.scala} | 20 +- .../measure/process/BatchDqProcess.scala | 4 +- .../{algo => process}/ProcessType.scala | 2 +- .../measure/process/StreamingDqProcess.scala | 5 +- .../measure/process/StreamingDqThread.scala | 2 +- .../streaming => process}/TimingProcess.scala | 2 +- .../process/engine/DataFrameOprEngine.scala | 4 +- .../measure/process/engine/DqEngine.scala | 4 +- .../measure/process/engine/DqEngines.scala | 4 +- .../process/engine/SparkDqEngine.scala | 4 +- .../process/engine/SparkSqlEngine.scala | 4 +- .../measure/rule/CalculationUtil.scala | 315 --------------- .../rule/DataTypeCalculationUtil.scala | 159 -------- .../griffin/measure/rule/ExprValueUtil.scala | 263 ------------- .../griffin/measure/rule/RuleAnalyzer.scala | 72 ---- .../griffin/measure/rule/RuleFactory.scala | 52 --- .../griffin/measure/rule/RuleParser.scala | 244 ------------ .../measure/rule/SchemaValueCombineUtil.scala | 187 --------- .../{rules => rule}/adaptor/AdaptPhase.scala | 2 +- .../adaptor/DataFrameOprAdaptor.scala | 6 +- .../adaptor/GriffinDslAdaptor.scala | 13 +- .../{rules => rule}/adaptor/RuleAdaptor.scala | 6 +- .../adaptor/RuleAdaptorGroup.scala | 8 +- .../adaptor/SparkSqlAdaptor.scala | 4 +- .../measure/{rules => rule}/dsl/DqType.scala | 2 +- .../measure/{rules => rule}/dsl/DslType.scala | 2 +- .../{rules => rule}/dsl/PersistType.scala | 2 +- .../dsl/analyzer/AccuracyAnalyzer.scala | 4 +- .../dsl/analyzer/BasicAnalyzer.scala | 4 +- .../dsl/analyzer/ProfilingAnalyzer.scala | 4 +- .../dsl/expr/AliasableExpr.scala | 2 +- .../dsl/expr/ClauseExpression.scala | 2 +- .../{rules => rule}/dsl/expr/Expr.scala | 2 +- .../dsl/expr/FunctionExpr.scala | 2 +- .../dsl/expr/LiteralExpr.scala | 2 +- .../dsl/expr/LogicalExpr.scala | 2 +- .../{rules => rule}/dsl/expr/MathExpr.scala | 2 +- .../{rules => rule}/dsl/expr/SelectExpr.scala | 2 +- .../{rules => rule}/dsl/expr/TreeNode.scala | 2 +- .../dsl/parser/BasicParser.scala | 4 +- .../dsl/parser/GriffinDslParser.scala | 6 +- .../measure/rule/expr/AnalyzableExpr.scala | 24 -- .../griffin/measure/rule/expr/Cacheable.scala | 33 -- .../measure/rule/expr/Calculatable.scala | 25 -- .../measure/rule/expr/ClauseExpr.scala | 109 ------ .../measure/rule/expr/DataSourceable.scala | 28 -- .../measure/rule/expr/Describable.scala | 33 -- .../griffin/measure/rule/expr/Expr.scala | 53 --- .../measure/rule/expr/ExprDescOnly.scala | 40 -- .../measure/rule/expr/ExprIdCounter.scala | 60 --- .../measure/rule/expr/FieldDescOnly.scala | 58 --- .../measure/rule/expr/LiteralExpr.scala | 83 ---- .../measure/rule/expr/LogicalExpr.scala | 178 --------- .../griffin/measure/rule/expr/MathExpr.scala | 99 ----- .../measure/rule/expr/SelectExpr.scala | 88 ----- .../rule/func/DefaultFunctionDefine.scala | 36 -- .../measure/rule/func/FunctionDefine.scala | 25 -- .../measure/rule/func/FunctionUtil.scala | 75 ---- .../preproc/PreProcRuleGenerator.scala | 2 +- .../step/ConcreteRuleStep.scala | 4 +- .../{rules => rule}/step/DfOprStep.scala | 4 +- .../{rules => rule}/step/GriffinDslStep.scala | 4 +- .../{rules => rule}/step/RuleStep.scala | 4 +- .../{rules => rule}/step/SparkSqlStep.scala | 4 +- .../{rules => rule}/udf/GriffinUdfs.scala | 2 +- .../config-test-profiling-streaming.json | 2 +- .../algo/batch/BatchAccuracyAlgoTest.scala | 294 -------------- .../algo/batch/BatchProfileAlgoTest.scala | 173 -------- .../algo/batch/DataFrameSaveTest.scala | 172 -------- .../measure/algo/core/AccuracyCoreTest.scala | 89 ----- .../measure/algo/core/ProfileCoreTest.scala | 79 ---- .../streaming/StreamingAccuracyAlgoTest.scala | 261 ------------- .../data/connector/ConnectorTest.scala | 4 +- .../measure/process/BatchProcessTest.scala | 4 +- .../process/StreamingProcessTest.scala | 4 +- .../measure/rule/ExprValueUtilTest.scala | 86 ---- .../measure/rule/RuleAnalyzerTest.scala | 60 --- .../measure/rule/RuleFactoryTest.scala | 44 --- .../griffin/measure/rule/RuleParserTest.scala | 213 ---------- .../dsl/parser/BasicParserTest.scala | 4 +- 92 files changed, 94 insertions(+), 5176 deletions(-) delete mode 100644 measure/src/main/scala/org/apache/griffin/measure/algo/AccuracyAlgo.scala delete mode 100644 measure/src/main/scala/org/apache/griffin/measure/algo/Algo.scala delete mode 100644 measure/src/main/scala/org/apache/griffin/measure/algo/MeasureType.scala delete mode 100644 measure/src/main/scala/org/apache/griffin/measure/algo/ProfileAlgo.scala delete mode 100644 measure/src/main/scala/org/apache/griffin/measure/algo/batch/BatchAccuracyAlgo.scala delete mode 100644 measure/src/main/scala/org/apache/griffin/measure/algo/batch/BatchProfileAlgo.scala delete mode 100644 measure/src/main/scala/org/apache/griffin/measure/algo/core/AccuracyCore.scala delete mode 100644 measure/src/main/scala/org/apache/griffin/measure/algo/core/ProfileCore.scala delete mode 100644 measure/src/main/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgo.scala delete mode 100644 measure/src/main/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyProcess.scala delete mode 100644 measure/src/main/scala/org/apache/griffin/measure/config/params/user/RuleParam.scala rename measure/src/main/scala/org/apache/griffin/measure/{config/params/user/DataCacheParam.scala => process/Algo.scala} (62%) rename measure/src/main/scala/org/apache/griffin/measure/{algo => process}/ProcessType.scala (97%) rename measure/src/main/scala/org/apache/griffin/measure/{algo/streaming => process}/TimingProcess.scala (96%) delete mode 100644 measure/src/main/scala/org/apache/griffin/measure/rule/CalculationUtil.scala delete mode 100644 measure/src/main/scala/org/apache/griffin/measure/rule/DataTypeCalculationUtil.scala delete mode 100644 measure/src/main/scala/org/apache/griffin/measure/rule/ExprValueUtil.scala delete mode 100644 measure/src/main/scala/org/apache/griffin/measure/rule/RuleAnalyzer.scala delete mode 100644 measure/src/main/scala/org/apache/griffin/measure/rule/RuleFactory.scala delete mode 100644 measure/src/main/scala/org/apache/griffin/measure/rule/RuleParser.scala delete mode 100644 measure/src/main/scala/org/apache/griffin/measure/rule/SchemaValueCombineUtil.scala rename measure/src/main/scala/org/apache/griffin/measure/{rules => rule}/adaptor/AdaptPhase.scala (94%) rename measure/src/main/scala/org/apache/griffin/measure/{rules => rule}/adaptor/DataFrameOprAdaptor.scala (90%) rename measure/src/main/scala/org/apache/griffin/measure/{rules => rule}/adaptor/GriffinDslAdaptor.scala (96%) rename measure/src/main/scala/org/apache/griffin/measure/{rules => rule}/adaptor/RuleAdaptor.scala (92%) rename measure/src/main/scala/org/apache/griffin/measure/{rules => rule}/adaptor/RuleAdaptorGroup.scala (95%) rename measure/src/main/scala/org/apache/griffin/measure/{rules => rule}/adaptor/SparkSqlAdaptor.scala (95%) rename measure/src/main/scala/org/apache/griffin/measure/{rules => rule}/dsl/DqType.scala (97%) rename measure/src/main/scala/org/apache/griffin/measure/{rules => rule}/dsl/DslType.scala (97%) rename measure/src/main/scala/org/apache/griffin/measure/{rules => rule}/dsl/PersistType.scala (97%) rename measure/src/main/scala/org/apache/griffin/measure/{rules => rule}/dsl/analyzer/AccuracyAnalyzer.scala (93%) rename measure/src/main/scala/org/apache/griffin/measure/{rules => rule}/dsl/analyzer/BasicAnalyzer.scala (94%) rename measure/src/main/scala/org/apache/griffin/measure/{rules => rule}/dsl/analyzer/ProfilingAnalyzer.scala (92%) rename measure/src/main/scala/org/apache/griffin/measure/{rules => rule}/dsl/expr/AliasableExpr.scala (94%) rename measure/src/main/scala/org/apache/griffin/measure/{rules => rule}/dsl/expr/ClauseExpression.scala (98%) rename measure/src/main/scala/org/apache/griffin/measure/{rules => rule}/dsl/expr/Expr.scala (94%) rename measure/src/main/scala/org/apache/griffin/measure/{rules => rule}/dsl/expr/FunctionExpr.scala (95%) rename measure/src/main/scala/org/apache/griffin/measure/{rules => rule}/dsl/expr/LiteralExpr.scala (97%) rename measure/src/main/scala/org/apache/griffin/measure/{rules => rule}/dsl/expr/LogicalExpr.scala (98%) rename measure/src/main/scala/org/apache/griffin/measure/{rules => rule}/dsl/expr/MathExpr.scala (97%) rename measure/src/main/scala/org/apache/griffin/measure/{rules => rule}/dsl/expr/SelectExpr.scala (98%) rename measure/src/main/scala/org/apache/griffin/measure/{rules => rule}/dsl/expr/TreeNode.scala (97%) rename measure/src/main/scala/org/apache/griffin/measure/{rules => rule}/dsl/parser/BasicParser.scala (99%) rename measure/src/main/scala/org/apache/griffin/measure/{rules => rule}/dsl/parser/GriffinDslParser.scala (88%) delete mode 100644 measure/src/main/scala/org/apache/griffin/measure/rule/expr/AnalyzableExpr.scala delete mode 100644 measure/src/main/scala/org/apache/griffin/measure/rule/expr/Cacheable.scala delete mode 100644 measure/src/main/scala/org/apache/griffin/measure/rule/expr/Calculatable.scala delete mode 100644 measure/src/main/scala/org/apache/griffin/measure/rule/expr/ClauseExpr.scala delete mode 100644 measure/src/main/scala/org/apache/griffin/measure/rule/expr/DataSourceable.scala delete mode 100644 measure/src/main/scala/org/apache/griffin/measure/rule/expr/Describable.scala delete mode 100644 measure/src/main/scala/org/apache/griffin/measure/rule/expr/Expr.scala delete mode 100644 measure/src/main/scala/org/apache/griffin/measure/rule/expr/ExprDescOnly.scala delete mode 100644 measure/src/main/scala/org/apache/griffin/measure/rule/expr/ExprIdCounter.scala delete mode 100644 measure/src/main/scala/org/apache/griffin/measure/rule/expr/FieldDescOnly.scala delete mode 100644 measure/src/main/scala/org/apache/griffin/measure/rule/expr/LiteralExpr.scala delete mode 100644 measure/src/main/scala/org/apache/griffin/measure/rule/expr/LogicalExpr.scala delete mode 100644 measure/src/main/scala/org/apache/griffin/measure/rule/expr/MathExpr.scala delete mode 100644 measure/src/main/scala/org/apache/griffin/measure/rule/expr/SelectExpr.scala delete mode 100644 measure/src/main/scala/org/apache/griffin/measure/rule/func/DefaultFunctionDefine.scala delete mode 100644 measure/src/main/scala/org/apache/griffin/measure/rule/func/FunctionDefine.scala delete mode 100644 measure/src/main/scala/org/apache/griffin/measure/rule/func/FunctionUtil.scala rename measure/src/main/scala/org/apache/griffin/measure/{rules => rule}/preproc/PreProcRuleGenerator.scala (97%) rename measure/src/main/scala/org/apache/griffin/measure/{rules => rule}/step/ConcreteRuleStep.scala (92%) rename measure/src/main/scala/org/apache/griffin/measure/{rules => rule}/step/DfOprStep.scala (91%) rename measure/src/main/scala/org/apache/griffin/measure/{rules => rule}/step/GriffinDslStep.scala (91%) rename measure/src/main/scala/org/apache/griffin/measure/{rules => rule}/step/RuleStep.scala (88%) rename measure/src/main/scala/org/apache/griffin/measure/{rules => rule}/step/SparkSqlStep.scala (92%) rename measure/src/main/scala/org/apache/griffin/measure/{rules => rule}/udf/GriffinUdfs.scala (95%) delete mode 100644 measure/src/test/scala/org/apache/griffin/measure/algo/batch/BatchAccuracyAlgoTest.scala delete mode 100644 measure/src/test/scala/org/apache/griffin/measure/algo/batch/BatchProfileAlgoTest.scala delete mode 100644 measure/src/test/scala/org/apache/griffin/measure/algo/batch/DataFrameSaveTest.scala delete mode 100644 measure/src/test/scala/org/apache/griffin/measure/algo/core/AccuracyCoreTest.scala delete mode 100644 measure/src/test/scala/org/apache/griffin/measure/algo/core/ProfileCoreTest.scala delete mode 100644 measure/src/test/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgoTest.scala delete mode 100644 measure/src/test/scala/org/apache/griffin/measure/rule/ExprValueUtilTest.scala delete mode 100644 measure/src/test/scala/org/apache/griffin/measure/rule/RuleAnalyzerTest.scala delete mode 100644 measure/src/test/scala/org/apache/griffin/measure/rule/RuleFactoryTest.scala delete mode 100644 measure/src/test/scala/org/apache/griffin/measure/rule/RuleParserTest.scala rename measure/src/test/scala/org/apache/griffin/measure/{rules => rule}/dsl/parser/BasicParserTest.scala (98%) diff --git a/measure/src/main/scala/org/apache/griffin/measure/algo/AccuracyAlgo.scala b/measure/src/main/scala/org/apache/griffin/measure/algo/AccuracyAlgo.scala deleted file mode 100644 index 7e0a56331..000000000 --- a/measure/src/main/scala/org/apache/griffin/measure/algo/AccuracyAlgo.scala +++ /dev/null @@ -1,24 +0,0 @@ -/* -Licensed to the Apache Software Foundation (ASF) under one -or more contributor license agreements. See the NOTICE file -distributed with this work for additional information -regarding copyright ownership. The ASF licenses this file -to you under the Apache License, Version 2.0 (the -"License"); you may not use this file except in compliance -with the License. You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, -software distributed under the License is distributed on an -"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -KIND, either express or implied. See the License for the -specific language governing permissions and limitations -under the License. -*/ -package org.apache.griffin.measure.algo - - -trait AccuracyAlgo extends Algo { - -} diff --git a/measure/src/main/scala/org/apache/griffin/measure/algo/Algo.scala b/measure/src/main/scala/org/apache/griffin/measure/algo/Algo.scala deleted file mode 100644 index 82b71f1f2..000000000 --- a/measure/src/main/scala/org/apache/griffin/measure/algo/Algo.scala +++ /dev/null @@ -1,34 +0,0 @@ -/* -Licensed to the Apache Software Foundation (ASF) under one -or more contributor license agreements. See the NOTICE file -distributed with this work for additional information -regarding copyright ownership. The ASF licenses this file -to you under the Apache License, Version 2.0 (the -"License"); you may not use this file except in compliance -with the License. You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, -software distributed under the License is distributed on an -"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -KIND, either express or implied. See the License for the -specific language governing permissions and limitations -under the License. -*/ -package org.apache.griffin.measure.algo - -import org.apache.griffin.measure.config.params.env._ -import org.apache.griffin.measure.config.params.user._ -import org.apache.griffin.measure.log.Loggable - -import scala.util.Try - -trait Algo extends Loggable with Serializable { - - val envParam: EnvParam - val userParam: UserParam - - def run(): Try[_] - -} diff --git a/measure/src/main/scala/org/apache/griffin/measure/algo/MeasureType.scala b/measure/src/main/scala/org/apache/griffin/measure/algo/MeasureType.scala deleted file mode 100644 index 23d4dac43..000000000 --- a/measure/src/main/scala/org/apache/griffin/measure/algo/MeasureType.scala +++ /dev/null @@ -1,26 +0,0 @@ -/* -Licensed to the Apache Software Foundation (ASF) under one -or more contributor license agreements. See the NOTICE file -distributed with this work for additional information -regarding copyright ownership. The ASF licenses this file -to you under the Apache License, Version 2.0 (the -"License"); you may not use this file except in compliance -with the License. You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, -software distributed under the License is distributed on an -"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -KIND, either express or implied. See the License for the -specific language governing permissions and limitations -under the License. -*/ -package org.apache.griffin.measure.algo - -object MeasureType { - - val accuracy = """^(?i)accuracy$""".r - val profile = """^(?i)profile$""".r - -} diff --git a/measure/src/main/scala/org/apache/griffin/measure/algo/ProfileAlgo.scala b/measure/src/main/scala/org/apache/griffin/measure/algo/ProfileAlgo.scala deleted file mode 100644 index 6ffc87aad..000000000 --- a/measure/src/main/scala/org/apache/griffin/measure/algo/ProfileAlgo.scala +++ /dev/null @@ -1,23 +0,0 @@ -/* -Licensed to the Apache Software Foundation (ASF) under one -or more contributor license agreements. See the NOTICE file -distributed with this work for additional information -regarding copyright ownership. The ASF licenses this file -to you under the Apache License, Version 2.0 (the -"License"); you may not use this file except in compliance -with the License. You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, -software distributed under the License is distributed on an -"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -KIND, either express or implied. See the License for the -specific language governing permissions and limitations -under the License. -*/ -package org.apache.griffin.measure.algo - -trait ProfileAlgo extends Algo { - -} diff --git a/measure/src/main/scala/org/apache/griffin/measure/algo/batch/BatchAccuracyAlgo.scala b/measure/src/main/scala/org/apache/griffin/measure/algo/batch/BatchAccuracyAlgo.scala deleted file mode 100644 index 5098d925d..000000000 --- a/measure/src/main/scala/org/apache/griffin/measure/algo/batch/BatchAccuracyAlgo.scala +++ /dev/null @@ -1,190 +0,0 @@ -///* -//Licensed to the Apache Software Foundation (ASF) under one -//or more contributor license agreements. See the NOTICE file -//distributed with this work for additional information -//regarding copyright ownership. The ASF licenses this file -//to you under the Apache License, Version 2.0 (the -//"License"); you may not use this file except in compliance -//with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -//Unless required by applicable law or agreed to in writing, -//software distributed under the License is distributed on an -//"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -//KIND, either express or implied. See the License for the -//specific language governing permissions and limitations -//under the License. -//*/ -//package org.apache.griffin.measure.algo.batch -// -//import java.util.Date -// -//import org.apache.griffin.measure.algo.AccuracyAlgo -//import org.apache.griffin.measure.algo.core.AccuracyCore -//import org.apache.griffin.measure.config.params.AllParam -//import org.apache.griffin.measure.data.connector._ -//import org.apache.griffin.measure.data.connector.direct.DirectDataConnector -//import org.apache.griffin.measure.persist._ -//import org.apache.griffin.measure.result._ -//import org.apache.griffin.measure.rule._ -//import org.apache.griffin.measure.rule.expr._ -//import org.apache.spark.rdd.RDD -//import org.apache.spark.sql.hive.HiveContext -//import org.apache.spark.{SparkConf, SparkContext} -// -//import scala.util.{Failure, Success, Try} -// -//// accuracy algorithm for batch mode -//case class BatchAccuracyAlgo(allParam: AllParam) extends AccuracyAlgo { -// val envParam = allParam.envParam -// val userParam = allParam.userParam -// -// def run(): Try[_] = { -// Try { -// val metricName = userParam.name -// -// val sparkParam = envParam.sparkParam -// -// val conf = new SparkConf().setAppName(metricName) -// conf.setAll(sparkParam.config) -// val sc = new SparkContext(conf) -// sc.setLogLevel(sparkParam.logLevel) -// val sqlContext = new HiveContext(sc) -// -// // start time -// val startTime = new Date().getTime() -// -// // get persists to persist measure result -// val persist: Persist = PersistFactory(envParam.persistParams, metricName).getPersists(startTime) -// -// // get spark application id -// val applicationId = sc.applicationId -// -// // persist start id -// persist.start(applicationId) -// -// // generate rule from rule param, generate rule analyzer -// val ruleFactory = RuleFactory(userParam.evaluateRuleParam) -// val rule: StatementExpr = ruleFactory.generateRule() -// val ruleAnalyzer: RuleAnalyzer = RuleAnalyzer(rule) -// -// // const expr value map -// val constExprValueMap = ExprValueUtil.genExprValueMaps(None, ruleAnalyzer.constCacheExprs, Map[String, Any]()) -// val finalConstExprValueMap = ExprValueUtil.updateExprValueMaps(ruleAnalyzer.constFinalCacheExprs, constExprValueMap) -// val finalConstMap = finalConstExprValueMap.headOption match { -// case Some(m) => m -// case _ => Map[String, Any]() -// } -// -// // data connector -// val sourceDataConnector: DirectDataConnector = -// DataConnectorFactory.getDirectDataConnector(sqlContext, null, userParam.sourceParam, -// ruleAnalyzer.sourceRuleExprs, finalConstMap -// ) match { -// case Success(cntr) => { -// if (cntr.available) cntr -// else throw new Exception("source data connection error!") -// } -// case Failure(ex) => throw ex -// } -// val targetDataConnector: DirectDataConnector = -// DataConnectorFactory.getDirectDataConnector(sqlContext, null, userParam.targetParam, -// ruleAnalyzer.targetRuleExprs, finalConstMap -// ) match { -// case Success(cntr) => { -// if (cntr.available) cntr -// else throw new Exception("target data connection error!") -// } -// case Failure(ex) => throw ex -// } -// -// // get metadata -//// val sourceMetaData: Iterable[(String, String)] = sourceDataConnector.metaData() match { -//// case Success(md) => md -//// case _ => throw new Exception("source metadata error!") -//// } -//// val targetMetaData: Iterable[(String, String)] = targetDataConnector.metaData() match { -//// case Success(md) => md -//// case _ => throw new Exception("target metadata error!") -//// } -// -// // get data -// val sourceData: RDD[(Product, (Map[String, Any], Map[String, Any]))] = sourceDataConnector.data() match { -// case Success(dt) => dt -// case Failure(ex) => throw ex -// } -// val targetData: RDD[(Product, (Map[String, Any], Map[String, Any]))] = targetDataConnector.data() match { -// case Success(dt) => dt -// case Failure(ex) => throw ex -// } -// -// // accuracy algorithm -// val (accuResult, missingRdd, matchedRdd) = accuracy(sourceData, targetData, ruleAnalyzer) -// -// // end time -// val endTime = new Date().getTime -// persist.log(endTime, s"calculation using time: ${endTime - startTime} ms") -// -// // persist result -// persist.result(endTime, accuResult) -// val missingRecords = missingRdd.map(record2String(_, ruleAnalyzer.sourceRuleExprs.persistExprs, ruleAnalyzer.targetRuleExprs.persistExprs)) -//// persist.missRecords(missingRecords) -// persist.records(missingRecords, PersistType.MISS) -// -// // persist end time -// val persistEndTime = new Date().getTime -// persist.log(persistEndTime, s"persist using time: ${persistEndTime - endTime} ms") -// -// // finish -// persist.finish() -// -// // context stop -// sc.stop -// -// } -// } -// -// def wrapInitData(data: Map[String, Any]): (Map[String, Any], Map[String, Any]) = { -// (data, Map[String, Any]()) -// } -// -// // calculate accuracy between source data and target data -// def accuracy(sourceData: RDD[(Product, (Map[String, Any], Map[String, Any]))], -// targetData: RDD[(Product, (Map[String, Any], Map[String, Any]))], -// ruleAnalyzer: RuleAnalyzer) = { -// // 1. cogroup -// val allKvs = sourceData.cogroup(targetData) -// -// // 2. accuracy calculation -// val (accuResult, missingRdd, matchedRdd) = AccuracyCore.accuracy(allKvs, ruleAnalyzer) -// -// (accuResult, missingRdd, matchedRdd) -// } -// -// // convert data into a string -// def record2String(rec: (Product, (Map[String, Any], Map[String, Any])), sourcePersist: Iterable[Expr], targetPersist: Iterable[Expr]): String = { -// val (key, (data, info)) = rec -// val persistData = getPersistMap(data, sourcePersist) -// val persistInfo = info.mapValues { value => -// value match { -// case vd: Map[String, Any] => getPersistMap(vd, targetPersist) -// case v => v -// } -// }.map(identity) -// s"${persistData} [${persistInfo}]" -// } -// -// // get the expr value map of the persist expressions -// private def getPersistMap(data: Map[String, Any], persist: Iterable[Expr]): Map[String, Any] = { -// val persistMap = persist.map(e => (e._id, e.desc)).toMap -// data.flatMap { pair => -// val (k, v) = pair -// persistMap.get(k) match { -// case Some(d) => Some((d -> v)) -// case _ => None -// } -// } -// } -// -//} diff --git a/measure/src/main/scala/org/apache/griffin/measure/algo/batch/BatchProfileAlgo.scala b/measure/src/main/scala/org/apache/griffin/measure/algo/batch/BatchProfileAlgo.scala deleted file mode 100644 index 53e764804..000000000 --- a/measure/src/main/scala/org/apache/griffin/measure/algo/batch/BatchProfileAlgo.scala +++ /dev/null @@ -1,162 +0,0 @@ -///* -//Licensed to the Apache Software Foundation (ASF) under one -//or more contributor license agreements. See the NOTICE file -//distributed with this work for additional information -//regarding copyright ownership. The ASF licenses this file -//to you under the Apache License, Version 2.0 (the -//"License"); you may not use this file except in compliance -//with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -//Unless required by applicable law or agreed to in writing, -//software distributed under the License is distributed on an -//"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -//KIND, either express or implied. See the License for the -//specific language governing permissions and limitations -//under the License. -//*/ -//package org.apache.griffin.measure.algo.batch -// -//import java.util.Date -// -//import org.apache.griffin.measure.algo.ProfileAlgo -//import org.apache.griffin.measure.algo.core.ProfileCore -//import org.apache.griffin.measure.config.params._ -//import org.apache.griffin.measure.data.connector._ -//import org.apache.griffin.measure.data.connector.direct.DirectDataConnector -//import org.apache.griffin.measure.persist.{Persist, PersistFactory, PersistType} -//import org.apache.griffin.measure.result._ -//import org.apache.griffin.measure.rule.expr._ -//import org.apache.griffin.measure.rule.{ExprValueUtil, RuleAnalyzer, RuleFactory} -//import org.apache.spark.rdd.RDD -//import org.apache.spark.sql.hive.HiveContext -//import org.apache.spark.{SparkConf, SparkContext} -// -//import scala.util.{Failure, Success, Try} -// -//// profile algorithm for batch mode -//case class BatchProfileAlgo(allParam: AllParam) extends ProfileAlgo { -// val envParam = allParam.envParam -// val userParam = allParam.userParam -// -// def run(): Try[_] = { -// Try { -// val metricName = userParam.name -// -// val sparkParam = envParam.sparkParam -// -// val conf = new SparkConf().setAppName(metricName) -// conf.setAll(sparkParam.config) -// val sc = new SparkContext(conf) -// sc.setLogLevel(sparkParam.logLevel) -// val sqlContext = new HiveContext(sc) -// -// // start time -// val startTime = new Date().getTime() -// -// // get persists to persist measure result -// val persist: Persist = PersistFactory(envParam.persistParams, metricName).getPersists(startTime) -// -// // get spark application id -// val applicationId = sc.applicationId -// -// // persist start id -// persist.start(applicationId) -// -// // generate rule from rule param, generate rule analyzer -// val ruleFactory = RuleFactory(userParam.evaluateRuleParam) -// val rule: StatementExpr = ruleFactory.generateRule() -// val ruleAnalyzer: RuleAnalyzer = RuleAnalyzer(rule) -// -// // const expr value map -// val constExprValueMap = ExprValueUtil.genExprValueMaps(None, ruleAnalyzer.constCacheExprs, Map[String, Any]()) -// val finalConstExprValueMap = ExprValueUtil.updateExprValueMaps(ruleAnalyzer.constFinalCacheExprs, constExprValueMap) -// val finalConstMap = finalConstExprValueMap.headOption match { -// case Some(m) => m -// case _ => Map[String, Any]() -// } -// -// // data connector -// val sourceDataConnector: DirectDataConnector = -// DataConnectorFactory.getDirectDataConnector(sqlContext, null, userParam.sourceParam, -// ruleAnalyzer.sourceRuleExprs, finalConstMap -// ) match { -// case Success(cntr) => { -// if (cntr.available) cntr -// else throw new Exception("source data connection error!") -// } -// case Failure(ex) => throw ex -// } -// -// // get metadata -// // val sourceMetaData: Iterable[(String, String)] = sourceDataConnector.metaData() match { -// // case Success(md) => md -// // case _ => throw new Exception("source metadata error!") -// // } -// -// // get data -// val sourceData: RDD[(Product, (Map[String, Any], Map[String, Any]))] = sourceDataConnector.data() match { -// case Success(dt) => dt -// case Failure(ex) => throw ex -// } -// -// // profile algorithm -// val (profileResult, missingRdd, matchedRdd) = profile(sourceData, ruleAnalyzer) -// -// // end time -// val endTime = new Date().getTime -// persist.log(endTime, s"calculation using time: ${endTime - startTime} ms") -// -// // persist result -// persist.result(endTime, profileResult) -// val matchedRecords = matchedRdd.map(record2String(_, ruleAnalyzer.sourceRuleExprs.persistExprs)) -//// persist.matchRecords(matchedRecords) -// persist.records(matchedRecords, PersistType.MATCH) -// -// // persist end time -// val persistEndTime = new Date().getTime -// persist.log(persistEndTime, s"persist using time: ${persistEndTime - endTime} ms") -// -// // finish -// persist.finish() -// -// // context stop -// sc.stop -// } -// } -// -// def wrapInitData(data: Map[String, Any]): (Map[String, Any], Map[String, Any]) = { -// (data, Map[String, Any]()) -// } -// -// // calculate profile from source data -// def profile(sourceData: RDD[(Product, (Map[String, Any], Map[String, Any]))], ruleAnalyzer: RuleAnalyzer -// ) = { -// // 1. profile calculation -// val (profileResult, missingRdd, matchedRdd) = ProfileCore.profile(sourceData, ruleAnalyzer) -// -// (profileResult, missingRdd, matchedRdd) -// } -// -// // convert data into a string -// def record2String(rec: (Product, (Map[String, Any], Map[String, Any])), sourcePersist: Iterable[Expr]): String = { -// val (key, (data, info)) = rec -// val persistData = getPersistMap(data, sourcePersist) -// val persistInfo = info -// if (persistInfo.size > 0) s"${persistData} [${persistInfo}]" else s"${persistData}" -// } -// -// // get the expr value map of the persist expressions -// private def getPersistMap(data: Map[String, Any], persist: Iterable[Expr]): Map[String, Any] = { -// val persistMap = persist.map(e => (e._id, e.desc)).toMap -// data.flatMap { pair => -// val (k, v) = pair -// persistMap.get(k) match { -// case Some(d) => Some((d -> v)) -// case _ => None -// } -// } -// } -// -//} diff --git a/measure/src/main/scala/org/apache/griffin/measure/algo/core/AccuracyCore.scala b/measure/src/main/scala/org/apache/griffin/measure/algo/core/AccuracyCore.scala deleted file mode 100644 index 4ec6505c8..000000000 --- a/measure/src/main/scala/org/apache/griffin/measure/algo/core/AccuracyCore.scala +++ /dev/null @@ -1,103 +0,0 @@ -/* -Licensed to the Apache Software Foundation (ASF) under one -or more contributor license agreements. See the NOTICE file -distributed with this work for additional information -regarding copyright ownership. The ASF licenses this file -to you under the Apache License, Version 2.0 (the -"License"); you may not use this file except in compliance -with the License. You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, -software distributed under the License is distributed on an -"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -KIND, either express or implied. See the License for the -specific language governing permissions and limitations -under the License. -*/ -package org.apache.griffin.measure.algo.core - -import org.apache.griffin.measure.rule.RuleAnalyzer -import org.apache.griffin.measure.result._ -import org.apache.spark.rdd.RDD - - -object AccuracyCore { - - type V = Map[String, Any] - type T = Map[String, Any] - - // allKvs: rdd of (key, (List[(sourceData, sourceInfo)], List[(targetData, targetInfo)])) - // output: accuracy result, missing source data rdd, matched source data rdd - def accuracy(allKvs: RDD[(Product, (Iterable[(V, T)], Iterable[(V, T)]))], ruleAnalyzer: RuleAnalyzer - ): (AccuracyResult, RDD[(Product, (V, T))], RDD[(Product, (V, T))]) = { - val result: RDD[(Long, Long, List[(Product, (V, T))], List[(Product, (V, T))])] = allKvs.map { kv => - val (key, (sourceDatas, targetDatas)) = kv - - // result: (missCount, matchCount, missDataList, matchDataList) - val rslt = sourceDatas.foldLeft((0L, 0L, List[(Product, (V, T))](), List[(Product, (V, T))]())) { (sr, sourcePair) => - val matchResult = if (targetDatas.isEmpty) { - (false, Map[String, Any](MismatchInfo.wrap("no target"))) - } else { - targetDatas.foldLeft((false, Map[String, Any]())) { (tr, targetPair) => - if (tr._1) tr - else matchData(sourcePair, targetPair, ruleAnalyzer) - } - } - - if (matchResult._1) { - val matchItem = (key, sourcePair) - (sr._1, sr._2 + 1, sr._3, sr._4 :+ matchItem) - } else { - val missItem = (key, (sourcePair._1, sourcePair._2 ++ matchResult._2)) - (sr._1 + 1, sr._2, sr._3 :+ missItem, sr._4) - } - } - - rslt - } - - val missRdd = result.flatMap(_._3) - val matchRdd = result.flatMap(_._4) - - def seq(cnt: (Long, Long), rcd: (Long, Long, Any, Any)): (Long, Long) = { - (cnt._1 + rcd._1, cnt._2 + rcd._2) - } - def comb(c1: (Long, Long), c2: (Long, Long)): (Long, Long) = { - (c1._1 + c2._1, c1._2 + c2._2) - } - val countPair = result.aggregate((0L, 0L))(seq, comb) - - (AccuracyResult(countPair._1, (countPair._1 + countPair._2)), missRdd, matchRdd) - } - - // try to match source and target data, return true if matched, false if unmatched, also with some matching info - private def matchData(source: (V, T), target: (V, T), ruleAnalyzer: RuleAnalyzer): (Boolean, T) = { - - // 1. merge source and target cached data - val mergedExprValueMap: Map[String, Any] = mergeExprValueMap(source, target) - - // 2. check valid - if (ruleAnalyzer.rule.valid(mergedExprValueMap)) { - // 3. substitute the cached data into statement, get the statement value - val matched = ruleAnalyzer.rule.calculate(mergedExprValueMap) match { - case Some(b: Boolean) => b - case _ => false - } - // currently we can not get the mismatch reason, we need to add such information to figure out how it mismatches - if (matched) (matched, Map[String, Any]()) - else (matched, Map[String, Any](MismatchInfo.wrap("not matched"))) - } else { - (false, Map[String, Any](MismatchInfo.wrap("invalid to compare"))) - } - - } - -// private def when - - private def mergeExprValueMap(source: (V, T), target: (V, T)): Map[String, Any] = { - source._1 ++ target._1 - } - -} diff --git a/measure/src/main/scala/org/apache/griffin/measure/algo/core/ProfileCore.scala b/measure/src/main/scala/org/apache/griffin/measure/algo/core/ProfileCore.scala deleted file mode 100644 index 2987f2fdf..000000000 --- a/measure/src/main/scala/org/apache/griffin/measure/algo/core/ProfileCore.scala +++ /dev/null @@ -1,73 +0,0 @@ -/* -Licensed to the Apache Software Foundation (ASF) under one -or more contributor license agreements. See the NOTICE file -distributed with this work for additional information -regarding copyright ownership. The ASF licenses this file -to you under the Apache License, Version 2.0 (the -"License"); you may not use this file except in compliance -with the License. You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, -software distributed under the License is distributed on an -"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -KIND, either express or implied. See the License for the -specific language governing permissions and limitations -under the License. -*/ -package org.apache.griffin.measure.algo.core - -import org.apache.griffin.measure.rule.RuleAnalyzer -import org.apache.griffin.measure.result._ -import org.apache.spark.rdd.RDD - - -object ProfileCore { - - type V = Map[String, Any] - type T = Map[String, Any] - - // dataRdd: rdd of (key, (sourceData, sourceInfo)) - // output: accuracy result, missing source data rdd, matched source data rdd - def profile(dataRdd: RDD[(Product, (V, T))], ruleAnalyzer: RuleAnalyzer - ): (ProfileResult, RDD[(Product, (V, T))], RDD[(Product, (V, T))]) = { - - val resultRdd: RDD[((Product, (V, T)), Boolean)] = dataRdd.map { kv => - val (key, (data, info)) = kv - val (matched, missInfo) = matchData((data, info), ruleAnalyzer) - ((key, (data, info ++ missInfo)), matched) - } - - val totalCount = resultRdd.count - val matchRdd = resultRdd.filter(_._2).map(_._1) - val matchCount = matchRdd.count - val missRdd = resultRdd.filter(!_._2).map(_._1) - val missCount = missRdd.count - - (ProfileResult(matchCount, totalCount), missRdd, matchRdd) - - } - - // try to match data as rule, return true if matched, false if unmatched - private def matchData(dataPair: (V, T), ruleAnalyzer: RuleAnalyzer): (Boolean, T) = { - - val data: Map[String, Any] = dataPair._1 - - // 1. check valid - if (ruleAnalyzer.rule.valid(data)) { - // 2. substitute the cached data into statement, get the statement value - val matched = ruleAnalyzer.rule.calculate(data) match { - case Some(b: Boolean) => b - case _ => false - } - // currently we can not get the mismatch reason, we need to add such information to figure out how it mismatches - if (matched) (matched, Map[String, Any]()) - else (matched, Map[String, Any](MismatchInfo.wrap("not matched"))) - } else { - (false, Map[String, Any](MismatchInfo.wrap("invalid to compare"))) - } - - } - -} diff --git a/measure/src/main/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgo.scala b/measure/src/main/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgo.scala deleted file mode 100644 index 9df17c364..000000000 --- a/measure/src/main/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgo.scala +++ /dev/null @@ -1,369 +0,0 @@ -///* -//Licensed to the Apache Software Foundation (ASF) under one -//or more contributor license agreements. See the NOTICE file -//distributed with this work for additional information -//regarding copyright ownership. The ASF licenses this file -//to you under the Apache License, Version 2.0 (the -//"License"); you may not use this file except in compliance -//with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -//Unless required by applicable law or agreed to in writing, -//software distributed under the License is distributed on an -//"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -//KIND, either express or implied. See the License for the -//specific language governing permissions and limitations -//under the License. -//*/ -//package org.apache.griffin.measure.algo.streaming -// -//import java.util.Date -//import java.util.concurrent.{Executors, ThreadPoolExecutor, TimeUnit} -// -//import org.apache.griffin.measure.algo.AccuracyAlgo -//import org.apache.griffin.measure.algo.core.AccuracyCore -//import org.apache.griffin.measure.cache.info.{InfoCacheInstance, TimeInfoCache} -//import org.apache.griffin.measure.cache.result.CacheResultProcesser -//import org.apache.griffin.measure.config.params.AllParam -//import org.apache.griffin.measure.data.connector._ -//import org.apache.griffin.measure.data.connector.direct.DirectDataConnector -//import org.apache.griffin.measure.persist.{Persist, PersistFactory, PersistType} -//import org.apache.griffin.measure.result.{AccuracyResult, MismatchInfo, TimeStampInfo} -//import org.apache.griffin.measure.rule.{ExprValueUtil, RuleAnalyzer, RuleFactory} -//import org.apache.griffin.measure.rule.expr._ -//import org.apache.griffin.measure.utils.TimeUtil -//import org.apache.spark.rdd.RDD -//import org.apache.spark.sql.SQLContext -//import org.apache.spark.sql.hive.HiveContext -//import org.apache.spark.streaming.{Milliseconds, Seconds, StreamingContext} -//import org.apache.spark.{SparkConf, SparkContext} -// -//import scala.util.{Failure, Success, Try} -// -// -//case class StreamingAccuracyAlgo(allParam: AllParam) extends AccuracyAlgo { -// val envParam = allParam.envParam -// val userParam = allParam.userParam -// -// def run(): Try[_] = { -// Try { -// val metricName = userParam.name -// -// val sparkParam = envParam.sparkParam -// -// val conf = new SparkConf().setAppName(metricName) -// conf.setAll(sparkParam.config) -// val sc = new SparkContext(conf) -// sc.setLogLevel(sparkParam.logLevel) -// val sqlContext = new HiveContext(sc) -//// val sqlContext = new SQLContext(sc) -// -//// val batchInterval = TimeUtil.milliseconds(sparkParam.batchInterval) match { -//// case Some(interval) => Milliseconds(interval) -//// case _ => throw new Exception("invalid batch interval") -//// } -//// val ssc = new StreamingContext(sc, batchInterval) -//// ssc.checkpoint(sparkParam.cpDir) -// -// def createStreamingContext(): StreamingContext = { -// val batchInterval = TimeUtil.milliseconds(sparkParam.batchInterval) match { -// case Some(interval) => Milliseconds(interval) -// case _ => throw new Exception("invalid batch interval") -// } -// val ssc = new StreamingContext(sc, batchInterval) -// ssc.checkpoint(sparkParam.cpDir) -// ssc -// } -// val ssc = StreamingContext.getOrCreate(sparkParam.cpDir, createStreamingContext) -// -// // init info cache instance -// InfoCacheInstance.initInstance(envParam.infoCacheParams, metricName) -// InfoCacheInstance.init -// -// // start time -// val startTime = new Date().getTime() -// -// val persistFactory = PersistFactory(envParam.persistParams, metricName) -// -// // get persists to persist measure result -// val appPersist: Persist = persistFactory.getPersists(startTime) -// -// // get spark application id -// val applicationId = sc.applicationId -// -// // persist start id -// appPersist.start(applicationId) -// -// // generate rule from rule param, generate rule analyzer -// val ruleFactory = RuleFactory(userParam.evaluateRuleParam) -// val rule: StatementExpr = ruleFactory.generateRule() -// val ruleAnalyzer: RuleAnalyzer = RuleAnalyzer(rule) -// -// // const expr value map -// val constExprValueMap = ExprValueUtil.genExprValueMaps(None, ruleAnalyzer.constCacheExprs, Map[String, Any]()) -// val finalConstExprValueMap = ExprValueUtil.updateExprValueMaps(ruleAnalyzer.constFinalCacheExprs, constExprValueMap) -// val finalConstMap = finalConstExprValueMap.headOption match { -// case Some(m) => m -// case _ => Map[String, Any]() -// } -// -// // data connector -// val sourceDataConnector: DirectDataConnector = -// DataConnectorFactory.getDirectDataConnector(sqlContext, ssc, userParam.sourceParam, -// ruleAnalyzer.sourceRuleExprs, finalConstMap -// ) match { -// case Success(cntr) => { -// if (cntr.available) cntr -// else throw new Exception("source data connection error!") -// } -// case Failure(ex) => throw ex -// } -// val targetDataConnector: DirectDataConnector = -// DataConnectorFactory.getDirectDataConnector(sqlContext, ssc, userParam.targetParam, -// ruleAnalyzer.targetRuleExprs, finalConstMap -// ) match { -// case Success(cntr) => { -// if (cntr.available) cntr -// else throw new Exception("target data connection error!") -// } -// case Failure(ex) => throw ex -// } -// -// val cacheResultProcesser = CacheResultProcesser() -// -// // init data stream -// sourceDataConnector.init() -// targetDataConnector.init() -// -// val streamingAccuracyProcess = StreamingAccuracyProcess( -// sourceDataConnector, targetDataConnector, -// ruleAnalyzer, cacheResultProcesser, persistFactory, appPersist) -// -// // process thread -//// case class Process() extends Runnable { -//// val lock = InfoCacheInstance.genLock("process") -//// def run(): Unit = { -//// val updateTime = new Date().getTime -//// val locked = lock.lock(5, TimeUnit.SECONDS) -//// if (locked) { -//// try { -//// val st = new Date().getTime -//// -//// TimeInfoCache.startTimeInfoCache -//// -//// // get data -//// val sourceData = sourceDataConnector.data match { -//// case Success(dt) => dt -//// case Failure(ex) => throw ex -//// } -//// val targetData = targetDataConnector.data match { -//// case Success(dt) => dt -//// case Failure(ex) => throw ex -//// } -//// -//// sourceData.cache -//// targetData.cache -//// -//// println(s"sourceData.count: ${sourceData.count}") -//// println(s"targetData.count: ${targetData.count}") -//// -//// // accuracy algorithm -//// val (accuResult, missingRdd, matchedRdd) = accuracy(sourceData, targetData, ruleAnalyzer) -//// println(s"accuResult: ${accuResult}") -//// -//// val ct = new Date().getTime -//// appPersist.log(ct, s"calculation using time: ${ct - st} ms") -//// -//// sourceData.unpersist() -//// targetData.unpersist() -//// -//// // result of every group -//// val matchedGroups = reorgByTimeGroup(matchedRdd) -//// val matchedGroupCount = matchedGroups.count -//// println(s"===== matchedGroupCount: ${matchedGroupCount} =====") -//// -//// // get missing results -//// val missingGroups = reorgByTimeGroup(missingRdd) -//// val missingGroupCount = missingGroups.count -//// println(s"===== missingGroupCount: ${missingGroupCount} =====") -//// -//// val groups = matchedGroups.cogroup(missingGroups) -//// val groupCount = groups.count -//// println(s"===== groupCount: ${groupCount} =====") -//// -//// val updateResults = groups.flatMap { group => -//// val (t, (matchData, missData)) = group -//// -//// val matchSize = matchData.size -//// val missSize = missData.size -//// val res = AccuracyResult(missSize, matchSize + missSize) -//// -//// val updatedCacheResultOpt = cacheResultProcesser.genUpdateCacheResult(t, updateTime, res) -//// -//// updatedCacheResultOpt.flatMap { updatedCacheResult => -//// Some((updatedCacheResult, (t, missData))) -//// } -//// } -//// -//// updateResults.cache -//// -//// val updateResultsPart = updateResults.map(_._1) -//// val updateDataPart = updateResults.map(_._2) -//// -//// val updateResultsArray = updateResultsPart.collect() -//// -//// // update results cache (in driver) -//// // collect action is traversable once action, it will make rdd updateResults empty -//// updateResultsArray.foreach { updateResult => -//// println(s"update result: ${updateResult}") -//// cacheResultProcesser.update(updateResult) -//// // persist result -//// val persist: Persist = persistFactory.getPersists(updateResult.timeGroup) -//// persist.result(updateTime, updateResult.result) -//// } -//// -//// // record missing data and update old data (in executor) -//// updateDataPart.foreach { grp => -//// val (t, datas) = grp -//// val persist: Persist = persistFactory.getPersists(t) -//// // persist missing data -//// val missStrings = datas.map { row => -//// val (_, (value, info)) = row -//// s"${value} [${info.getOrElse(MismatchInfo.key, "unknown")}]" -//// } -//// persist.records(missStrings, PersistType.MISS) -//// // data connector update old data -//// val dumpDatas = datas.map { r => -//// val (_, (v, i)) = r -//// v ++ i -//// } -//// -//// println(t) -//// dumpDatas.foreach(println) -//// -//// sourceDataConnector.updateOldData(t, dumpDatas) -//// targetDataConnector.updateOldData(t, dumpDatas) // not correct -//// } -//// -//// updateResults.unpersist() -//// -//// // dump missing rdd (this part not need for future version, only for current df cache data version) -//// val dumpRdd: RDD[Map[String, Any]] = missingRdd.map { r => -//// val (_, (v, i)) = r -//// v ++ i -//// } -//// sourceDataConnector.updateAllOldData(dumpRdd) -//// targetDataConnector.updateAllOldData(dumpRdd) // not correct -//// -//// TimeInfoCache.endTimeInfoCache -//// -//// val et = new Date().getTime -//// appPersist.log(et, s"persist using time: ${et - ct} ms") -//// -//// } catch { -//// case e: Throwable => error(s"process error: ${e.getMessage}") -//// } finally { -//// lock.unlock() -//// } -//// } -//// } -//// } -// -// val processInterval = TimeUtil.milliseconds(sparkParam.processInterval) match { -// case Some(interval) => interval -// case _ => throw new Exception("invalid batch interval") -// } -// val process = TimingProcess(processInterval, streamingAccuracyProcess) -// -// // clean thread -//// case class Clean() extends Runnable { -//// val lock = InfoCacheInstance.genLock("clean") -//// def run(): Unit = { -//// val locked = lock.lock(5, TimeUnit.SECONDS) -//// if (locked) { -//// try { -//// sourceDataConnector.cleanData -//// targetDataConnector.cleanData -//// } finally { -//// lock.unlock() -//// } -//// } -//// } -//// } -//// val cleanInterval = TimeUtil.milliseconds(cleanerParam.cleanInterval) match { -//// case Some(interval) => interval -//// case _ => throw new Exception("invalid batch interval") -//// } -//// val clean = TimingProcess(cleanInterval, Clean()) -// -// process.startup() -//// clean.startup() -// -// ssc.start() -// ssc.awaitTermination() -// ssc.stop(stopSparkContext=true, stopGracefully=true) -// -// // context stop -// sc.stop -// -// InfoCacheInstance.close -// -// appPersist.finish() -// -// process.shutdown() -//// clean.shutdown() -// } -// } -// -// // calculate accuracy between source data and target data -//// def accuracy(sourceData: RDD[(Product, (Map[String, Any], Map[String, Any]))], -//// targetData: RDD[(Product, (Map[String, Any], Map[String, Any]))], -//// ruleAnalyzer: RuleAnalyzer) = { -//// // 1. cogroup -//// val allKvs = sourceData.cogroup(targetData) -//// -//// // 2. accuracy calculation -//// val (accuResult, missingRdd, matchedRdd) = AccuracyCore.accuracy(allKvs, ruleAnalyzer) -//// -//// (accuResult, missingRdd, matchedRdd) -//// } -// -//// // convert data into a string -//// def record2String(rec: (Product, (Map[String, Any], Map[String, Any])), sourcePersist: Iterable[Expr], targetPersist: Iterable[Expr]): String = { -//// val (key, (data, info)) = rec -//// val persistData = getPersistMap(data, sourcePersist) -//// val persistInfo = info.mapValues { value => -//// value match { -//// case vd: Map[String, Any] => getPersistMap(vd, targetPersist) -//// case v => v -//// } -//// }.map(identity) -//// s"${persistData} [${persistInfo}]" -//// } -//// -//// // get the expr value map of the persist expressions -//// private def getPersistMap(data: Map[String, Any], persist: Iterable[Expr]): Map[String, Any] = { -//// val persistMap = persist.map(e => (e._id, e.desc)).toMap -//// data.flatMap { pair => -//// val (k, v) = pair -//// persistMap.get(k) match { -//// case Some(d) => Some((d -> v)) -//// case _ => None -//// } -//// } -//// } -// -//// def reorgByTimeGroup(rdd: RDD[(Product, (Map[String, Any], Map[String, Any]))] -//// ): RDD[(Long, (Product, (Map[String, Any], Map[String, Any])))] = { -//// rdd.flatMap { row => -//// val (key, (value, info)) = row -//// val b: Option[(Long, (Product, (Map[String, Any], Map[String, Any])))] = info.get(TimeStampInfo.key) match { -//// case Some(t: Long) => Some((t, row)) -//// case _ => None -//// } -//// b -//// } -//// } -// -//} diff --git a/measure/src/main/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyProcess.scala b/measure/src/main/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyProcess.scala deleted file mode 100644 index 1ca8d3449..000000000 --- a/measure/src/main/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyProcess.scala +++ /dev/null @@ -1,234 +0,0 @@ -///* -//Licensed to the Apache Software Foundation (ASF) under one -//or more contributor license agreements. See the NOTICE file -//distributed with this work for additional information -//regarding copyright ownership. The ASF licenses this file -//to you under the Apache License, Version 2.0 (the -//"License"); you may not use this file except in compliance -//with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -//Unless required by applicable law or agreed to in writing, -//software distributed under the License is distributed on an -//"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -//KIND, either express or implied. See the License for the -//specific language governing permissions and limitations -//under the License. -//*/ -//package org.apache.griffin.measure.algo.streaming -// -//import java.util.Date -//import java.util.concurrent.TimeUnit -// -//import org.apache.griffin.measure.algo.core.AccuracyCore -//import org.apache.griffin.measure.cache.info.{InfoCacheInstance, TimeInfoCache} -//import org.apache.griffin.measure.cache.result.CacheResultProcesser -//import org.apache.griffin.measure.data.connector.direct.DirectDataConnector -//import org.apache.griffin.measure.log.Loggable -//import org.apache.griffin.measure.persist._ -//import org.apache.griffin.measure.result.{AccuracyResult, MismatchInfo, TimeStampInfo} -//import org.apache.griffin.measure.rule._ -//import org.apache.griffin.measure.rule.expr._ -//import org.apache.spark.rdd.RDD -// -//import scala.util.{Failure, Success} -// -//case class StreamingAccuracyProcess(sourceDataConnector: DirectDataConnector, -// targetDataConnector: DirectDataConnector, -// ruleAnalyzer: RuleAnalyzer, -// cacheResultProcesser: CacheResultProcesser, -// persistFactory: PersistFactory, -// appPersist: Persist -// ) extends Runnable with Loggable { -// -// val lock = InfoCacheInstance.genLock("process") -// -// def run(): Unit = { -//// println(s"cache count: ${cacheResultProcesser.cacheGroup.size}") -// val updateTimeDate = new Date() -// val updateTime = updateTimeDate.getTime -// println(s"===== [${updateTimeDate}] process begins =====") -// val locked = lock.lock(5, TimeUnit.SECONDS) -// if (locked) { -// try { -// val st = new Date().getTime -// -// TimeInfoCache.startTimeInfoCache -// -// // get data -// val sourceData = sourceDataConnector.data match { -// case Success(dt) => dt -// case Failure(ex) => throw ex -// } -// val targetData = targetDataConnector.data match { -// case Success(dt) => dt -// case Failure(ex) => throw ex -// } -// -// sourceData.cache -// targetData.cache -// -// println(s"sourceData.count: ${sourceData.count}") -// println(s"targetData.count: ${targetData.count}") -// -// // accuracy algorithm -// val (accuResult, missingRdd, matchedRdd) = accuracy(sourceData, targetData, ruleAnalyzer) -//// println(s"accuResult: ${accuResult}") -// -// val ct = new Date().getTime -// appPersist.log(ct, s"calculation using time: ${ct - st} ms") -// -// sourceData.unpersist() -// targetData.unpersist() -// -// // result of every group -// val matchedGroups = reorgByTimeGroup(matchedRdd) -//// val matchedGroupCount = matchedGroups.count -//// println(s"===== matchedGroupCount: ${matchedGroupCount} =====") -// -// // get missing results -// val missingGroups = reorgByTimeGroup(missingRdd) -//// val missingGroupCount = missingGroups.count -//// println(s"===== missingGroupCount: ${missingGroupCount} =====") -// -// val groups = matchedGroups.cogroup(missingGroups) -//// val groupCount = groups.count -//// println(s"===== groupCount: ${groupCount} =====") -// -// val updateResults = groups.flatMap { group => -// val (t, (matchData, missData)) = group -// -// val matchSize = matchData.size -// val missSize = missData.size -// val res = AccuracyResult(missSize, matchSize + missSize) -// -// val updatedCacheResultOpt = cacheResultProcesser.genUpdateCacheResult(t, updateTime, res) -// -// updatedCacheResultOpt.flatMap { updatedCacheResult => -// Some((updatedCacheResult, (t, missData))) -// } -// } -// -// updateResults.cache -// -// val updateResultsPart = updateResults.map(_._1) -// val updateDataPart = updateResults.map(_._2) -// -// val updateResultsArray = updateResultsPart.collect() -// -// // update results cache (in driver) -// // collect action is traversable once action, it will make rdd updateResults empty -// updateResultsArray.foreach { updateResult => -//// println(s"update result: ${updateResult}") -// cacheResultProcesser.update(updateResult) -// // persist result -// val persist: Persist = persistFactory.getPersists(updateResult.timeGroup) -// persist.result(updateTime, updateResult.result) -// } -// -// // record missing data and dump old data (in executor) -// updateDataPart.foreach { grp => -// val (t, datas) = grp -// val persist: Persist = persistFactory.getPersists(t) -// // persist missing data -// val missStrings = datas.map { row => -// record2String(row, ruleAnalyzer.sourceRuleExprs.persistExprs, ruleAnalyzer.targetRuleExprs.persistExprs) -// } -// persist.records(missStrings, PersistDataType.MISS) -// -// // data connector update old data -// val dumpDatas = datas.map { r => -// val (_, (v, i)) = r -// v ++ i -// } -// sourceDataConnector.updateOldData(t, dumpDatas) -//// targetDataConnector.updateOldData(t, dumpDatas) // not correct -// } -// -// updateResults.unpersist() -// -// TimeInfoCache.endTimeInfoCache -// -// // clean old data -// cleanData() -// -// val et = new Date().getTime -// appPersist.log(et, s"persist using time: ${et - ct} ms") -// -// } catch { -// case e: Throwable => error(s"process error: ${e.getMessage}") -// } finally { -// lock.unlock() -// } -// } else { -// println(s"===== [${updateTimeDate}] process ignores =====") -// } -// val endTime = new Date().getTime -// println(s"===== [${updateTimeDate}] process ends, using ${endTime - updateTime} ms =====") -// } -// -// // clean old data and old result cache -// def cleanData(): Unit = { -// try { -// sourceDataConnector.cleanOldData -// targetDataConnector.cleanOldData -// -// val cleanTime = TimeInfoCache.getCleanTime -// cacheResultProcesser.refresh(cleanTime) -// } catch { -// case e: Throwable => error(s"clean data error: ${e.getMessage}") -// } -// } -// -// // calculate accuracy between source data and target data -// private def accuracy(sourceData: RDD[(Product, (Map[String, Any], Map[String, Any]))], -// targetData: RDD[(Product, (Map[String, Any], Map[String, Any]))], -// ruleAnalyzer: RuleAnalyzer) = { -// // 1. cogroup -// val allKvs = sourceData.cogroup(targetData) -// -// // 2. accuracy calculation -// val (accuResult, missingRdd, matchedRdd) = AccuracyCore.accuracy(allKvs, ruleAnalyzer) -// -// (accuResult, missingRdd, matchedRdd) -// } -// -// private def reorgByTimeGroup(rdd: RDD[(Product, (Map[String, Any], Map[String, Any]))] -// ): RDD[(Long, (Product, (Map[String, Any], Map[String, Any])))] = { -// rdd.flatMap { row => -// val (key, (value, info)) = row -// val b: Option[(Long, (Product, (Map[String, Any], Map[String, Any])))] = info.get(TimeStampInfo.key) match { -// case Some(t: Long) => Some((t, row)) -// case _ => None -// } -// b -// } -// } -// -// // convert data into a string -// def record2String(rec: (Product, (Map[String, Any], Map[String, Any])), dataPersist: Iterable[Expr], infoPersist: Iterable[Expr]): String = { -// val (key, (data, info)) = rec -// val persistData = getPersistMap(data, dataPersist) -// val persistInfo = info.mapValues { value => -// value match { -// case vd: Map[String, Any] => getPersistMap(vd, infoPersist) -// case v => v -// } -// }.map(identity) -// s"${persistData} [${persistInfo}]" -// } -// -// // get the expr value map of the persist expressions -// private def getPersistMap(data: Map[String, Any], persist: Iterable[Expr]): Map[String, Any] = { -// val persistMap = persist.map(e => (e._id, e.desc)).toMap -// data.flatMap { pair => -// val (k, v) = pair -// persistMap.get(k) match { -// case Some(d) => Some((d -> v)) -// case _ => None -// } -// } -// } -// -//} diff --git a/measure/src/main/scala/org/apache/griffin/measure/config/params/user/RuleParam.scala b/measure/src/main/scala/org/apache/griffin/measure/config/params/user/RuleParam.scala deleted file mode 100644 index 696f4fe20..000000000 --- a/measure/src/main/scala/org/apache/griffin/measure/config/params/user/RuleParam.scala +++ /dev/null @@ -1,34 +0,0 @@ -///* -//Licensed to the Apache Software Foundation (ASF) under one -//or more contributor license agreements. See the NOTICE file -//distributed with this work for additional information -//regarding copyright ownership. The ASF licenses this file -//to you under the Apache License, Version 2.0 (the -//"License"); you may not use this file except in compliance -//with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -//Unless required by applicable law or agreed to in writing, -//software distributed under the License is distributed on an -//"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -//KIND, either express or implied. See the License for the -//specific language governing permissions and limitations -//under the License. -//*/ -//package org.apache.griffin.measure.config.params.user -// -//import com.fasterxml.jackson.annotation.{JsonInclude, JsonProperty} -//import com.fasterxml.jackson.annotation.JsonInclude.Include -//import org.apache.griffin.measure.config.params.Param -// -//@JsonInclude(Include.NON_NULL) -//case class RuleParam(@JsonProperty("name") name: String, -// @JsonProperty("dsl.type") dslType: String, -// @JsonProperty("rule") rule: String, -// @JsonProperty("persist.type") persistType: String, -// @JsonProperty("dq.type") dqType: String, -// @JsonProperty("details") details: Map[String, Any] -// ) extends Param { -// -//} diff --git a/measure/src/main/scala/org/apache/griffin/measure/data/connector/DataConnector.scala b/measure/src/main/scala/org/apache/griffin/measure/data/connector/DataConnector.scala index 9a6ccb2ca..a4fb56bac 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/data/connector/DataConnector.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/data/connector/DataConnector.scala @@ -24,9 +24,9 @@ import org.apache.griffin.measure.algo._ import org.apache.griffin.measure.config.params.user.DataConnectorParam import org.apache.griffin.measure.log.Loggable import org.apache.griffin.measure.process.engine._ -import org.apache.griffin.measure.rules.adaptor.{PreProcPhase, RuleAdaptorGroup, RunPhase} -import org.apache.griffin.measure.rules.dsl._ -import org.apache.griffin.measure.rules.preproc.PreProcRuleGenerator +import org.apache.griffin.measure.rule.adaptor.{PreProcPhase, RuleAdaptorGroup, RunPhase} +import org.apache.griffin.measure.rule.dsl._ +import org.apache.griffin.measure.rule.preproc.PreProcRuleGenerator import org.apache.spark.rdd.RDD import org.apache.spark.sql.functions._ import org.apache.spark.sql.{DataFrame, SQLContext} diff --git a/measure/src/main/scala/org/apache/griffin/measure/config/params/user/DataCacheParam.scala b/measure/src/main/scala/org/apache/griffin/measure/process/Algo.scala similarity index 62% rename from measure/src/main/scala/org/apache/griffin/measure/config/params/user/DataCacheParam.scala rename to measure/src/main/scala/org/apache/griffin/measure/process/Algo.scala index f94fdb61b..7f1b153d6 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/config/params/user/DataCacheParam.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/process/Algo.scala @@ -16,15 +16,19 @@ //specific language governing permissions and limitations //under the License. //*/ -//package org.apache.griffin.measure.config.params.user +//package org.apache.griffin.measure.algo // -//import com.fasterxml.jackson.annotation.{JsonInclude, JsonProperty} -//import com.fasterxml.jackson.annotation.JsonInclude.Include -//import org.apache.griffin.measure.config.params.Param +//import org.apache.griffin.measure.config.params.env._ +//import org.apache.griffin.measure.config.params.user._ +//import org.apache.griffin.measure.log.Loggable // -//@JsonInclude(Include.NON_NULL) -//case class DataCacheParam( @JsonProperty("config") config: Map[String, Any], -// @JsonProperty("time.range") timeRange: List[String] -// ) extends Param { +//import scala.util.Try +// +//trait Algo extends Loggable with Serializable { +// +// val envParam: EnvParam +// val userParam: UserParam +// +// def run(): Try[_] // //} diff --git a/measure/src/main/scala/org/apache/griffin/measure/process/BatchDqProcess.scala b/measure/src/main/scala/org/apache/griffin/measure/process/BatchDqProcess.scala index c3b796620..029f6a91a 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/process/BatchDqProcess.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/process/BatchDqProcess.scala @@ -27,8 +27,8 @@ import org.apache.griffin.measure.config.params.user._ import org.apache.griffin.measure.data.source.DataSourceFactory import org.apache.griffin.measure.persist.{Persist, PersistFactory} import org.apache.griffin.measure.process.engine.{DqEngineFactory, SparkSqlEngine} -import org.apache.griffin.measure.rules.adaptor.{RuleAdaptorGroup, RunPhase} -import org.apache.griffin.measure.rules.udf.GriffinUdfs +import org.apache.griffin.measure.rule.adaptor.{RuleAdaptorGroup, RunPhase} +import org.apache.griffin.measure.rule.udf.GriffinUdfs import org.apache.griffin.measure.utils.JsonUtil import org.apache.spark.sql.SQLContext import org.apache.spark.sql.hive.HiveContext diff --git a/measure/src/main/scala/org/apache/griffin/measure/algo/ProcessType.scala b/measure/src/main/scala/org/apache/griffin/measure/process/ProcessType.scala similarity index 97% rename from measure/src/main/scala/org/apache/griffin/measure/algo/ProcessType.scala rename to measure/src/main/scala/org/apache/griffin/measure/process/ProcessType.scala index 2f2dc7ac6..36f88e159 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/algo/ProcessType.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/process/ProcessType.scala @@ -16,7 +16,7 @@ KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License. */ -package org.apache.griffin.measure.algo +package org.apache.griffin.measure.process import scala.util.matching.Regex diff --git a/measure/src/main/scala/org/apache/griffin/measure/process/StreamingDqProcess.scala b/measure/src/main/scala/org/apache/griffin/measure/process/StreamingDqProcess.scala index 2acd3c61e..a56794119 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/process/StreamingDqProcess.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/process/StreamingDqProcess.scala @@ -20,7 +20,6 @@ package org.apache.griffin.measure.process import java.util.Date -import org.apache.griffin.measure.algo.streaming.TimingProcess import org.apache.griffin.measure.cache.info.InfoCacheInstance import org.apache.griffin.measure.config.params._ import org.apache.griffin.measure.config.params.env._ @@ -28,8 +27,8 @@ import org.apache.griffin.measure.config.params.user._ import org.apache.griffin.measure.data.source.DataSourceFactory import org.apache.griffin.measure.persist.{Persist, PersistFactory} import org.apache.griffin.measure.process.engine.DqEngineFactory -import org.apache.griffin.measure.rules.adaptor.RuleAdaptorGroup -import org.apache.griffin.measure.rules.udf.GriffinUdfs +import org.apache.griffin.measure.rule.adaptor.RuleAdaptorGroup +import org.apache.griffin.measure.rule.udf.GriffinUdfs import org.apache.griffin.measure.utils.TimeUtil import org.apache.spark.sql.SQLContext import org.apache.spark.sql.hive.HiveContext diff --git a/measure/src/main/scala/org/apache/griffin/measure/process/StreamingDqThread.scala b/measure/src/main/scala/org/apache/griffin/measure/process/StreamingDqThread.scala index b83cb4ab0..c631b726d 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/process/StreamingDqThread.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/process/StreamingDqThread.scala @@ -29,7 +29,7 @@ import org.apache.griffin.measure.data.source.DataSource import org.apache.griffin.measure.log.Loggable import org.apache.griffin.measure.persist.{Persist, PersistFactory} import org.apache.griffin.measure.process.engine.DqEngines -import org.apache.griffin.measure.rules.adaptor.{RuleAdaptorGroup, RunPhase} +import org.apache.griffin.measure.rule.adaptor.{RuleAdaptorGroup, RunPhase} case class StreamingDqThread(dqEngines: DqEngines, dataSources: Seq[DataSource], diff --git a/measure/src/main/scala/org/apache/griffin/measure/algo/streaming/TimingProcess.scala b/measure/src/main/scala/org/apache/griffin/measure/process/TimingProcess.scala similarity index 96% rename from measure/src/main/scala/org/apache/griffin/measure/algo/streaming/TimingProcess.scala rename to measure/src/main/scala/org/apache/griffin/measure/process/TimingProcess.scala index e5bd7deff..8d9bcb27c 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/algo/streaming/TimingProcess.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/process/TimingProcess.scala @@ -16,7 +16,7 @@ KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License. */ -package org.apache.griffin.measure.algo.streaming +package org.apache.griffin.measure.process import java.util.concurrent.{Executors, ThreadPoolExecutor, TimeUnit} import java.util.{Timer, TimerTask} diff --git a/measure/src/main/scala/org/apache/griffin/measure/process/engine/DataFrameOprEngine.scala b/measure/src/main/scala/org/apache/griffin/measure/process/engine/DataFrameOprEngine.scala index dd685f227..b409b8de9 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/process/engine/DataFrameOprEngine.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/process/engine/DataFrameOprEngine.scala @@ -26,8 +26,8 @@ import org.apache.griffin.measure.data.connector.GroupByColumn import org.apache.griffin.measure.data.source.{DataSource, DataSourceFactory} import org.apache.griffin.measure.persist.{Persist, PersistFactory} import org.apache.griffin.measure.result.AccuracyResult -import org.apache.griffin.measure.rules.dsl._ -import org.apache.griffin.measure.rules.step._ +import org.apache.griffin.measure.rule.dsl._ +import org.apache.griffin.measure.rule.step._ import org.apache.griffin.measure.utils.JsonUtil import org.apache.spark.rdd.RDD import org.apache.spark.sql.types.{LongType, StringType, StructField, StructType} diff --git a/measure/src/main/scala/org/apache/griffin/measure/process/engine/DqEngine.scala b/measure/src/main/scala/org/apache/griffin/measure/process/engine/DqEngine.scala index 383d48a38..9510309d1 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/process/engine/DqEngine.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/process/engine/DqEngine.scala @@ -22,8 +22,8 @@ import org.apache.griffin.measure.config.params.user.DataSourceParam import org.apache.griffin.measure.data.source.DataSource import org.apache.griffin.measure.log.Loggable import org.apache.griffin.measure.persist.{Persist, PersistFactory} -import org.apache.griffin.measure.rules.dsl._ -import org.apache.griffin.measure.rules.step._ +import org.apache.griffin.measure.rule.dsl._ +import org.apache.griffin.measure.rule.step._ import org.apache.spark.rdd.RDD import org.apache.spark.sql.DataFrame diff --git a/measure/src/main/scala/org/apache/griffin/measure/process/engine/DqEngines.scala b/measure/src/main/scala/org/apache/griffin/measure/process/engine/DqEngines.scala index 361599a97..9f581a5ab 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/process/engine/DqEngines.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/process/engine/DqEngines.scala @@ -23,8 +23,8 @@ import org.apache.griffin.measure.data.connector.GroupByColumn import org.apache.griffin.measure.data.source._ import org.apache.griffin.measure.log.Loggable import org.apache.griffin.measure.persist.{Persist, PersistFactory} -import org.apache.griffin.measure.rules.dsl._ -import org.apache.griffin.measure.rules.step._ +import org.apache.griffin.measure.rule.dsl._ +import org.apache.griffin.measure.rule.step._ import org.apache.spark.rdd.RDD import org.apache.spark.sql.DataFrame diff --git a/measure/src/main/scala/org/apache/griffin/measure/process/engine/SparkDqEngine.scala b/measure/src/main/scala/org/apache/griffin/measure/process/engine/SparkDqEngine.scala index a4b1b6c43..e00d65951 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/process/engine/SparkDqEngine.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/process/engine/SparkDqEngine.scala @@ -20,8 +20,8 @@ package org.apache.griffin.measure.process.engine import org.apache.griffin.measure.data.connector.GroupByColumn import org.apache.griffin.measure.log.Loggable -import org.apache.griffin.measure.rules.dsl.{MetricPersistType, RecordPersistType} -import org.apache.griffin.measure.rules.step._ +import org.apache.griffin.measure.rule.dsl.{MetricPersistType, RecordPersistType} +import org.apache.griffin.measure.rule.step._ import org.apache.griffin.measure.utils.JsonUtil import org.apache.spark.sql.{DataFrame, SQLContext} diff --git a/measure/src/main/scala/org/apache/griffin/measure/process/engine/SparkSqlEngine.scala b/measure/src/main/scala/org/apache/griffin/measure/process/engine/SparkSqlEngine.scala index ef745cb44..15df3b51b 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/process/engine/SparkSqlEngine.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/process/engine/SparkSqlEngine.scala @@ -24,8 +24,8 @@ import org.apache.griffin.measure.config.params.user.DataSourceParam import org.apache.griffin.measure.data.connector.GroupByColumn import org.apache.griffin.measure.data.source._ import org.apache.griffin.measure.persist.{Persist, PersistFactory} -import org.apache.griffin.measure.rules.dsl._ -import org.apache.griffin.measure.rules.step._ +import org.apache.griffin.measure.rule.dsl._ +import org.apache.griffin.measure.rule.step._ import org.apache.griffin.measure.utils.JsonUtil import org.apache.spark.rdd.RDD import org.apache.spark.sql.{DataFrame, GroupedData, SQLContext} diff --git a/measure/src/main/scala/org/apache/griffin/measure/rule/CalculationUtil.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/CalculationUtil.scala deleted file mode 100644 index c96901280..000000000 --- a/measure/src/main/scala/org/apache/griffin/measure/rule/CalculationUtil.scala +++ /dev/null @@ -1,315 +0,0 @@ -/* -Licensed to the Apache Software Foundation (ASF) under one -or more contributor license agreements. See the NOTICE file -distributed with this work for additional information -regarding copyright ownership. The ASF licenses this file -to you under the Apache License, Version 2.0 (the -"License"); you may not use this file except in compliance -with the License. You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, -software distributed under the License is distributed on an -"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -KIND, either express or implied. See the License for the -specific language governing permissions and limitations -under the License. -*/ -package org.apache.griffin.measure.rule - -import scala.util.{Success, Try} - - -object CalculationUtil { - - implicit def option2CalculationValue(v: Option[_]): CalculationValue = CalculationValue(v) - - // redefine the calculation method of operators in DSL - case class CalculationValue(value: Option[_]) extends Serializable { - - def + (other: Option[_]): Option[_] = { - Try { - (value, other) match { - case (None, _) | (_, None) => None - case (Some(null), _) | (_, Some(null)) => None - case (Some(v1: String), Some(v2)) => Some(v1 + v2.toString) - case (Some(v1: Byte), Some(v2)) => Some(v1 + v2.toString.toByte) - case (Some(v1: Short), Some(v2)) => Some(v1 + v2.toString.toShort) - case (Some(v1: Int), Some(v2)) => Some(v1 + v2.toString.toInt) - case (Some(v1: Long), Some(v2)) => Some(v1 + v2.toString.toLong) - case (Some(v1: Float), Some(v2)) => Some(v1 + v2.toString.toFloat) - case (Some(v1: Double), Some(v2)) => Some(v1 + v2.toString.toDouble) - case _ => value - } - } match { - case Success(opt) => opt - case _ => None - } - } - - def - (other: Option[_]): Option[_] = { - Try { - (value, other) match { - case (None, _) | (_, None) => None - case (Some(null), _) | (_, Some(null)) => None - case (Some(v1: Byte), Some(v2)) => Some(v1 - v2.toString.toByte) - case (Some(v1: Short), Some(v2)) => Some(v1 - v2.toString.toShort) - case (Some(v1: Int), Some(v2)) => Some(v1 - v2.toString.toInt) - case (Some(v1: Long), Some(v2)) => Some(v1 - v2.toString.toLong) - case (Some(v1: Float), Some(v2)) => Some(v1 - v2.toString.toFloat) - case (Some(v1: Double), Some(v2)) => Some(v1 - v2.toString.toDouble) - case _ => value - } - } match { - case Success(opt) => opt - case _ => None - } - } - - def * (other: Option[_]): Option[_] = { - Try { - (value, other) match { - case (None, _) | (_, None) => None - case (Some(null), _) | (_, Some(null)) => None - case (Some(s1: String), Some(n2: Int)) => Some(s1 * n2) - case (Some(s1: String), Some(n2: Long)) => Some(s1 * n2.toInt) - case (Some(v1: Byte), Some(v2)) => Some(v1 * v2.toString.toByte) - case (Some(v1: Short), Some(v2)) => Some(v1 * v2.toString.toShort) - case (Some(v1: Int), Some(v2)) => Some(v1 * v2.toString.toInt) - case (Some(v1: Long), Some(v2)) => Some(v1 * v2.toString.toLong) - case (Some(v1: Float), Some(v2)) => Some(v1 * v2.toString.toFloat) - case (Some(v1: Double), Some(v2)) => Some(v1 * v2.toString.toDouble) - case _ => value - } - } match { - case Success(opt) => opt - case _ => None - } - } - - def / (other: Option[_]): Option[_] = { - Try { - (value, other) match { - case (None, _) | (_, None) => None - case (Some(null), _) | (_, Some(null)) => None - case (Some(v1: Byte), Some(v2)) => Some(v1 / v2.toString.toByte) - case (Some(v1: Short), Some(v2)) => Some(v1 / v2.toString.toShort) - case (Some(v1: Int), Some(v2)) => Some(v1 / v2.toString.toInt) - case (Some(v1: Long), Some(v2)) => Some(v1 / v2.toString.toLong) - case (Some(v1: Float), Some(v2)) => Some(v1 / v2.toString.toFloat) - case (Some(v1: Double), Some(v2)) => Some(v1 / v2.toString.toDouble) - case _ => value - } - } match { - case Success(opt) => opt - case _ => None - } - } - - def % (other: Option[_]): Option[_] = { - Try { - (value, other) match { - case (None, _) | (_, None) => None - case (Some(null), _) | (_, Some(null)) => None - case (Some(v1: Byte), Some(v2)) => Some(v1 % v2.toString.toByte) - case (Some(v1: Short), Some(v2)) => Some(v1 % v2.toString.toShort) - case (Some(v1: Int), Some(v2)) => Some(v1 % v2.toString.toInt) - case (Some(v1: Long), Some(v2)) => Some(v1 % v2.toString.toLong) - case (Some(v1: Float), Some(v2)) => Some(v1 % v2.toString.toFloat) - case (Some(v1: Double), Some(v2)) => Some(v1 % v2.toString.toDouble) - case _ => value - } - } match { - case Success(opt) => opt - case _ => None - } - } - - def unary_- (): Option[_] = { - value match { - case None => None - case Some(null) => None - case Some(v: String) => Some(v.reverse.toString) - case Some(v: Boolean) => Some(!v) - case Some(v: Byte) => Some(-v) - case Some(v: Short) => Some(-v) - case Some(v: Int) => Some(-v) - case Some(v: Long) => Some(-v) - case Some(v: Float) => Some(-v) - case Some(v: Double) => Some(-v) - case Some(v) => Some(v) - case _ => None - } - } - - - def === (other: Option[_]): Option[Boolean] = { - (value, other) match { - case (None, None) => Some(true) - case (Some(v1), Some(v2)) => Some(v1 == v2) - case _ => Some(false) - } - } - - def =!= (other: Option[_]): Option[Boolean] = { - (value, other) match { - case (None, None) => Some(false) - case (Some(v1), Some(v2)) => Some(v1 != v2) - case _ => Some(true) - } - } - - def > (other: Option[_]): Option[Boolean] = { - Try { - (value, other) match { - case (None, _) | (_, None) => None - case (Some(null), _) | (_, Some(null)) => None - case (Some(v1: String), Some(v2: String)) => Some(v1 > v2) - case (Some(v1: Byte), Some(v2)) => Some(v1 > v2.toString.toDouble) - case (Some(v1: Short), Some(v2)) => Some(v1 > v2.toString.toDouble) - case (Some(v1: Int), Some(v2)) => Some(v1 > v2.toString.toDouble) - case (Some(v1: Long), Some(v2)) => Some(v1 > v2.toString.toDouble) - case (Some(v1: Float), Some(v2)) => Some(v1 > v2.toString.toDouble) - case (Some(v1: Double), Some(v2)) => Some(v1 > v2.toString.toDouble) - case _ => None - } - } match { - case Success(opt) => opt - case _ => None - } - } - - def >= (other: Option[_]): Option[Boolean] = { - Try { - (value, other) match { - case (None, None) | (Some(null), Some(null)) => Some(true) - case (None, _) | (_, None) => None - case (Some(null), _) | (_, Some(null)) => None - case (Some(v1: String), Some(v2: String)) => Some(v1 >= v2) - case (Some(v1: Byte), Some(v2)) => Some(v1 >= v2.toString.toDouble) - case (Some(v1: Short), Some(v2)) => Some(v1 >= v2.toString.toDouble) - case (Some(v1: Int), Some(v2)) => Some(v1 >= v2.toString.toDouble) - case (Some(v1: Long), Some(v2)) => Some(v1 >= v2.toString.toDouble) - case (Some(v1: Float), Some(v2)) => Some(v1 >= v2.toString.toDouble) - case (Some(v1: Double), Some(v2)) => Some(v1 >= v2.toString.toDouble) - case _ => None - } - } match { - case Success(opt) => opt - case _ => None - } - } - - def < (other: Option[_]): Option[Boolean] = { - Try { - (value, other) match { - case (None, _) | (_, None) => None - case (Some(null), _) | (_, Some(null)) => None - case (Some(v1: String), Some(v2: String)) => Some(v1 < v2) - case (Some(v1: Byte), Some(v2)) => Some(v1 < v2.toString.toDouble) - case (Some(v1: Short), Some(v2)) => Some(v1 < v2.toString.toDouble) - case (Some(v1: Int), Some(v2)) => Some(v1 < v2.toString.toDouble) - case (Some(v1: Long), Some(v2)) => Some(v1 < v2.toString.toDouble) - case (Some(v1: Float), Some(v2)) => Some(v1 < v2.toString.toDouble) - case (Some(v1: Double), Some(v2)) => Some(v1 < v2.toString.toDouble) - case _ => None - } - } match { - case Success(opt) => opt - case _ => None - } - } - - def <= (other: Option[_]): Option[Boolean] = { - Try { - (value, other) match { - case (None, None) | (Some(null), Some(null)) => Some(true) - case (None, _) | (_, None) => None - case (Some(null), _) | (_, Some(null)) => None - case (Some(v1: String), Some(v2: String)) => Some(v1 <= v2) - case (Some(v1: Byte), Some(v2)) => Some(v1 <= v2.toString.toDouble) - case (Some(v1: Short), Some(v2)) => Some(v1 <= v2.toString.toDouble) - case (Some(v1: Int), Some(v2)) => Some(v1 <= v2.toString.toDouble) - case (Some(v1: Long), Some(v2)) => Some(v1 <= v2.toString.toDouble) - case (Some(v1: Float), Some(v2)) => Some(v1 <= v2.toString.toDouble) - case (Some(v1: Double), Some(v2)) => Some(v1 <= v2.toString.toDouble) - case _ => None - } - } match { - case Success(opt) => opt - case _ => None - } - } - - - def in (other: Iterable[Option[_]]): Option[Boolean] = { - other.foldLeft(Some(false): Option[Boolean]) { (res, next) => - optOr(res, ===(next)) - } - } - - def not_in (other: Iterable[Option[_]]): Option[Boolean] = { - other.foldLeft(Some(true): Option[Boolean]) { (res, next) => - optAnd(res, =!=(next)) - } - } - - def between (other: Iterable[Option[_]]): Option[Boolean] = { - if (other.size < 2) None else { - val (begin, end) = (other.head, other.tail.head) - if (begin.isEmpty && end.isEmpty) Some(value.isEmpty) - else optAnd(>=(begin), <=(end)) - } - } - - def not_between (other: Iterable[Option[_]]): Option[Boolean] = { - if (other.size < 2) None else { - val (begin, end) = (other.head, other.tail.head) - if (begin.isEmpty && end.isEmpty) Some(value.nonEmpty) - else optOr(<(begin), >(end)) - } - } - - def unary_! (): Option[Boolean] = { - optNot(value) - } - - def && (other: Option[_]): Option[Boolean] = { - optAnd(value, other) - } - - def || (other: Option[_]): Option[Boolean] = { - optOr(value, other) - } - - - private def optNot(a: Option[_]): Option[Boolean] = { - a match { - case None => None - case Some(null) => None - case Some(v: Boolean) => Some(!v) - case _ => None - } - } - private def optAnd(a: Option[_], b: Option[_]): Option[Boolean] = { - (a, b) match { - case (None, _) | (_, None) => None - case (Some(null), _) | (_, Some(null)) => None - case (Some(false), _) | (_, Some(false)) => Some(false) - case (Some(b1: Boolean), Some(b2: Boolean)) => Some(b1 && b2) - case _ => None - } - } - private def optOr(a: Option[_], b: Option[_]): Option[Boolean] = { - (a, b) match { - case (None, _) | (_, None) => None - case (Some(null), _) | (_, Some(null)) => None - case (Some(true), _) | (_, Some(true)) => Some(true) - case (Some(b1: Boolean), Some(b2: Boolean)) => Some(b1 || b2) - case _ => None - } - } - } - -} diff --git a/measure/src/main/scala/org/apache/griffin/measure/rule/DataTypeCalculationUtil.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/DataTypeCalculationUtil.scala deleted file mode 100644 index 9d027ec5b..000000000 --- a/measure/src/main/scala/org/apache/griffin/measure/rule/DataTypeCalculationUtil.scala +++ /dev/null @@ -1,159 +0,0 @@ -/* -Licensed to the Apache Software Foundation (ASF) under one -or more contributor license agreements. See the NOTICE file -distributed with this work for additional information -regarding copyright ownership. The ASF licenses this file -to you under the Apache License, Version 2.0 (the -"License"); you may not use this file except in compliance -with the License. You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, -software distributed under the License is distributed on an -"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -KIND, either express or implied. See the License for the -specific language governing permissions and limitations -under the License. -*/ -package org.apache.griffin.measure.rule - -import org.apache.spark.sql.types._ - -object DataTypeCalculationUtil { - - implicit def dataType2CalculationType(tp: DataType): CalculationType = CalculationType(tp) - - case class CalculationType(tp: DataType) extends Serializable { - def binaryOpr (other: DataType): DataType = { - (tp, other) match { - case (NullType, _) | (_, NullType) => NullType - case (t, _) => t - } - } - def unaryOpr (): DataType = { - tp - } - } - - case class DataTypeException() extends Exception {} - - def getDataType(value: Any): DataType = { - value match { - case v: String => StringType - case v: Boolean => BooleanType - case v: Long => LongType - case v: Int => IntegerType - case v: Short => ShortType - case v: Byte => ByteType - case v: Double => DoubleType - case v: Float => FloatType - case v: Map[_, _] => MapType(getSameDataType(v.keys), getSameDataType(v.values)) - case v: Iterable[_] => ArrayType(getSameDataType(v)) - case _ => NullType - } - } - - private def getSameDataType(values: Iterable[Any]): DataType = { - values.foldLeft(NullType: DataType)((a, c) => genericTypeOf(a, getDataType(c))) - } - - private def genericTypeOf(dt1: DataType, dt2: DataType): DataType = { - if (dt1 == dt2) dt1 else { - dt1 match { - case NullType => dt2 - case StringType => dt1 - case DoubleType => { - dt2 match { - case StringType => dt2 - case DoubleType | FloatType | LongType | IntegerType | ShortType | ByteType => dt1 - case _ => throw DataTypeException() - } - } - case FloatType => { - dt2 match { - case StringType | DoubleType => dt2 - case FloatType | LongType | IntegerType | ShortType | ByteType => dt1 - case _ => throw DataTypeException() - } - } - case LongType => { - dt2 match { - case StringType | DoubleType | FloatType => dt2 - case LongType | IntegerType | ShortType | ByteType => dt1 - case _ => throw DataTypeException() - } - } - case IntegerType => { - dt2 match { - case StringType | DoubleType | FloatType | LongType => dt2 - case IntegerType | ShortType | ByteType => dt1 - case _ => throw DataTypeException() - } - } - case ShortType => { - dt2 match { - case StringType | DoubleType | FloatType | LongType | IntegerType => dt2 - case ShortType | ByteType => dt1 - case _ => throw DataTypeException() - } - } - case ByteType => { - dt2 match { - case StringType | DoubleType | FloatType | LongType | IntegerType | ShortType => dt2 - case ByteType => dt1 - case _ => throw DataTypeException() - } - } - case BooleanType => { - dt2 match { - case StringType => dt2 - case BooleanType => dt1 - case _ => throw DataTypeException() - } - } - case MapType(kdt1, vdt1, _) => { - dt2 match { - case MapType(kdt2, vdt2, _) => MapType(genericTypeOf(kdt1, kdt2), genericTypeOf(vdt1, vdt2)) - case _ => throw DataTypeException() - } - } - case ArrayType(vdt1, _) => { - dt2 match { - case ArrayType(vdt2, _) => ArrayType(genericTypeOf(vdt1, vdt2)) - case _ => throw DataTypeException() - } - } - case _ => throw DataTypeException() - } - } - } - - def sequenceDataTypeMap(aggr: Map[String, DataType], value: Map[String, Any]): Map[String, DataType] = { - val dataTypes = value.foldLeft(Map[String, DataType]()) { (map, pair) => - val (k, v) = pair - try { - map + (k -> getDataType(v)) - } catch { - case e: DataTypeException => map - } - } - combineDataTypeMap(aggr, dataTypes) - } - - def combineDataTypeMap(aggr1: Map[String, DataType], aggr2: Map[String, DataType]): Map[String, DataType] = { - aggr2.foldLeft(aggr1) { (a, c) => - a.get(c._1) match { - case Some(t) => { - try { - a + (c._1 -> genericTypeOf(t, c._2)) - } catch { - case e: DataTypeException => a - } - } - case _ => a + c - } - } - } - -} diff --git a/measure/src/main/scala/org/apache/griffin/measure/rule/ExprValueUtil.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/ExprValueUtil.scala deleted file mode 100644 index 940d0cb5f..000000000 --- a/measure/src/main/scala/org/apache/griffin/measure/rule/ExprValueUtil.scala +++ /dev/null @@ -1,263 +0,0 @@ -/* -Licensed to the Apache Software Foundation (ASF) under one -or more contributor license agreements. See the NOTICE file -distributed with this work for additional information -regarding copyright ownership. The ASF licenses this file -to you under the Apache License, Version 2.0 (the -"License"); you may not use this file except in compliance -with the License. You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, -software distributed under the License is distributed on an -"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -KIND, either express or implied. See the License for the -specific language governing permissions and limitations -under the License. -*/ -package org.apache.griffin.measure.rule - -import org.apache.griffin.measure.rule.expr._ -import org.apache.griffin.measure.rule.func._ -import org.apache.spark.sql.Row - -import scala.util.{Success, Try} - -object ExprValueUtil { - - private def append(path: List[String], step: String): List[String] = { - path :+ step - } - - private def value2Map(key: String, value: Option[Any]): Map[String, Any] = { - value.flatMap(v => Some((key -> v))).toMap - } - - private def getSingleValue(data: Option[Any], desc: FieldDescOnly): Option[Any] = { - data match { - case Some(row: Row) => { - desc match { - case i: IndexDesc => try { Some(row.getAs[Any](i.index)) } catch { case _ => None } - case f: FieldDesc => try { Some(row.getAs[Any](f.field)) } catch { case _ => None } - case _ => None - } - } - case Some(d: Map[String, Any]) => { - desc match { - case f: FieldDesc => d.get(f.field) - case _ => None - } - } - case Some(d: Seq[Any]) => { - desc match { - case i: IndexDesc => if (i.index >= 0 && i.index < d.size) Some(d(i.index)) else None - case _ => None - } - } - } - } - - private def calcExprValues(pathDatas: List[(List[String], Option[Any])], expr: Expr, existExprValueMap: Map[String, Any]): List[(List[String], Option[Any])] = { - Try { - expr match { - case selection: SelectionExpr => { - selection.selectors.foldLeft(pathDatas) { (pds, selector) => - calcExprValues(pds, selector, existExprValueMap) - } - } - case selector: IndexFieldRangeSelectExpr => { - pathDatas.flatMap { pathData => - val (path, data) = pathData - data match { - case Some(row: Row) => { - selector.fields.flatMap { field => - field match { - case (_: IndexDesc) | (_: FieldDesc) => { - getSingleValue(data, field).map { v => (append(path, field.desc), Some(v)) } - } - case a: AllFieldsDesc => { - (0 until row.size).flatMap { i => - getSingleValue(data, IndexDesc(i.toString)).map { v => - (append(path, s"${a.desc}_${i}"), Some(v)) - } - }.toList - } - case r: FieldRangeDesc => { - (r.startField, r.endField) match { - case (si: IndexDesc, ei: IndexDesc) => { - (si.index to ei.index).flatMap { i => - (append(path, s"${r.desc}_${i}"), getSingleValue(data, IndexDesc(i.toString))) - getSingleValue(data, IndexDesc(i.toString)).map { v => - (append(path, s"${r.desc}_${i}"), Some(v)) - } - }.toList - } - case _ => Nil - } - } - case _ => Nil - } - } - } - case Some(d: Map[String, Any]) => { - selector.fields.flatMap { field => - field match { - case (_: IndexDesc) | (_: FieldDesc) => { - getSingleValue(data, field).map { v => (append(path, field.desc), Some(v)) } - } - case a: AllFieldsDesc => { - d.keySet.flatMap { k => - getSingleValue(data, FieldDesc(k)).map { v => - (append(path, s"${a.desc}_${k}"), Some(v)) - } - } - } - case _ => None - } - } - } - case Some(d: Seq[Any]) => { - selector.fields.flatMap { field => - field match { - case (_: IndexDesc) | (_: FieldDesc) => { - getSingleValue(data, field).map { v => (append(path, field.desc), Some(v)) } - } - case a: AllFieldsDesc => { - (0 until d.size).flatMap { i => - (append(path, s"${a.desc}_${i}"), getSingleValue(data, IndexDesc(i.toString))) - getSingleValue(data, IndexDesc(i.toString)).map { v => - (append(path, s"${a.desc}_${i}"), Some(v)) - } - }.toList - } - case r: FieldRangeDesc => { - (r.startField, r.endField) match { - case (si: IndexDesc, ei: IndexDesc) => { - (si.index to ei.index).flatMap { i => - (append(path, s"${r.desc}_${i}"), getSingleValue(data, IndexDesc(i.toString))) - getSingleValue(data, IndexDesc(i.toString)).map { v => - (append(path, s"${r.desc}_${i}"), Some(v)) - } - }.toList - } - case _ => None - } - } - case _ => None - } - } - } - } - } - } - case selector: FunctionOperationExpr => { - val args: Array[Option[Any]] = selector.args.map { arg => - arg.calculate(existExprValueMap) - }.toArray - pathDatas.flatMap { pathData => - val (path, data) = pathData - data match { - case Some(d: String) => { - val res = FunctionUtil.invoke(selector.func, Some(d) +: args) - val residx = res.zipWithIndex - residx.map { vi => - val (v, i) = vi - val step = if (i == 0) s"${selector.desc}" else s"${selector.desc}_${i}" - (append(path, step), v) - } - } - case _ => None - } - } - } - case selector: FilterSelectExpr => { // fileter means select the items fit the condition - pathDatas.flatMap { pathData => - val (path, data) = pathData - data match { - case Some(row: Row) => { - // right value could not be selection - val rmap = value2Map(selector.value._id, selector.value.calculate(existExprValueMap)) - (0 until row.size).flatMap { i => - val dt = getSingleValue(data, IndexDesc(i.toString)) - val lmap = value2Map(selector.fieldKey, getSingleValue(dt, selector.field)) - val partValueMap = lmap ++ rmap - selector.calculate(partValueMap) match { - case Some(true) => Some((append(path, s"${selector.desc}_${i}"), dt)) - case _ => None - } - } - } - case Some(d: Map[String, Any]) => { - val rmap = value2Map(selector.value._id, selector.value.calculate(existExprValueMap)) - d.keySet.flatMap { k => - val dt = getSingleValue(data, FieldDesc(k)) - val lmap = value2Map(selector.fieldKey, getSingleValue(dt, selector.field)) - val partValueMap = lmap ++ rmap - selector.calculate(partValueMap) match { - case Some(true) => Some((append(path, s"${selector.desc}_${k}"), dt)) - case _ => None - } - } - } - case Some(d: Seq[Any]) => { - val rmap = value2Map(selector.value._id, selector.value.calculate(existExprValueMap)) - (0 until d.size).flatMap { i => - val dt = getSingleValue(data, IndexDesc(i.toString)) - val lmap = value2Map(selector.fieldKey, getSingleValue(dt, selector.field)) - val partValueMap = lmap ++ rmap - selector.calculate(partValueMap) match { - case Some(true) => Some((append(path, s"${selector.desc}_${i}"), dt)) - case _ => None - } - } - } - } - } - } - case _ => { - (expr.desc :: Nil, expr.calculate(existExprValueMap)) :: Nil - } - } - } match { - case Success(v) => v - case _ => Nil - } - } - - private def calcExprsValues(data: Option[Any], exprs: Iterable[Expr], existExprValueMap: Map[String, Any]): List[Map[String, Any]] = { - val selectionValues: Map[String, List[(List[String], Any)]] = exprs.map { expr => - (expr._id, calcExprValues((Nil, data) :: Nil, expr, existExprValueMap).flatMap { pair => - pair._2 match { - case Some(v) => Some((pair._1, v)) - case _ => None - } - }) - }.toMap - // if exprs is empty, return an empty value map for each row - if (selectionValues.isEmpty) List(Map[String, Any]()) - else SchemaValueCombineUtil.cartesian(selectionValues) - } - - // try to calculate some exprs from data and initExprValueMap, generate a new expression value map - // depends on origin data and existed expr value map - def genExprValueMaps(data: Option[Any], exprs: Iterable[Expr], initExprValueMap: Map[String, Any]): List[Map[String, Any]] = { - val (selections, nonSelections) = exprs.partition(_.isInstanceOf[SelectionExpr]) - val valueMaps = calcExprsValues(data, selections, initExprValueMap) - updateExprValueMaps(nonSelections, valueMaps) - } - - // with exprValueMap, calculate expressions, update the expression value map - // only depends on existed expr value map, only calculation, not need origin data - def updateExprValueMaps(exprs: Iterable[Expr], exprValueMaps: List[Map[String, Any]]): List[Map[String, Any]] = { - exprValueMaps.map { valueMap => - exprs.foldLeft(valueMap) { (em, expr) => - expr.calculate(em) match { - case Some(v) => em + (expr._id -> v) - case _ => em - } - } - } - } - -} diff --git a/measure/src/main/scala/org/apache/griffin/measure/rule/RuleAnalyzer.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/RuleAnalyzer.scala deleted file mode 100644 index 5ec143fb6..000000000 --- a/measure/src/main/scala/org/apache/griffin/measure/rule/RuleAnalyzer.scala +++ /dev/null @@ -1,72 +0,0 @@ -/* -Licensed to the Apache Software Foundation (ASF) under one -or more contributor license agreements. See the NOTICE file -distributed with this work for additional information -regarding copyright ownership. The ASF licenses this file -to you under the Apache License, Version 2.0 (the -"License"); you may not use this file except in compliance -with the License. You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, -software distributed under the License is distributed on an -"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -KIND, either express or implied. See the License for the -specific language governing permissions and limitations -under the License. -*/ -package org.apache.griffin.measure.rule - -import org.apache.griffin.measure.rule.expr._ - -case class RuleAnalyzer(rule: StatementExpr) extends Serializable { - - val constData = "" - private val SourceData = "source" - private val TargetData = "target" - - val constCacheExprs: Iterable[Expr] = rule.getCacheExprs(constData) - private val sourceCacheExprs: Iterable[Expr] = rule.getCacheExprs(SourceData) - private val targetCacheExprs: Iterable[Expr] = rule.getCacheExprs(TargetData) - - private val sourcePersistExprs: Iterable[Expr] = rule.getPersistExprs(SourceData) - private val targetPersistExprs: Iterable[Expr] = rule.getPersistExprs(TargetData) - - val constFinalCacheExprs: Iterable[Expr] = rule.getFinalCacheExprs(constData).toSet - private val sourceFinalCacheExprs: Iterable[Expr] = rule.getFinalCacheExprs(SourceData).toSet ++ sourcePersistExprs.toSet - private val targetFinalCacheExprs: Iterable[Expr] = rule.getFinalCacheExprs(TargetData).toSet ++ targetPersistExprs.toSet - - private val groupbyExprPairs: Seq[(Expr, Expr)] = rule.getGroupbyExprPairs((SourceData, TargetData)) - private val sourceGroupbyExprs: Seq[Expr] = groupbyExprPairs.map(_._1) - private val targetGroupbyExprs: Seq[Expr] = groupbyExprPairs.map(_._2) - - val sourceRuleExprs: RuleExprs = RuleExprs(sourceGroupbyExprs, sourceCacheExprs, - sourceFinalCacheExprs, sourcePersistExprs) - val targetRuleExprs: RuleExprs = RuleExprs(targetGroupbyExprs, targetCacheExprs, - targetFinalCacheExprs, targetPersistExprs) - -} - - -// for a single data source -// groupbyExprs: in accuracy case, these exprs could be groupby exprs -// Data keys for accuracy case, generated by the equal statements, to improve the calculation efficiency -// cacheExprs: the exprs value could be caculated independently, and cached for later use -// Cached for the finalCacheExprs calculation, it has some redundant values, saving it wastes a lot -// finalCacheExprs: the root of cachedExprs, cached for later use, plus with persistExprs -// Cached for the calculation usage, and can be saved for the re-calculation in streaming mode -// persistExprs: the expr values should be persisted, only the direct selection exprs are persistable -// Persisted for record usage, to record the missing data, need be readable as raw data -case class RuleExprs(groupbyExprs: Seq[Expr], - cacheExprs: Iterable[Expr], - finalCacheExprs: Iterable[Expr], - persistExprs: Iterable[Expr] - ) extends Serializable { - // for example: for a rule "$source.name = $target.name AND $source.age < $target.age + (3 * 4)" - // in this rule, for the target data source, the targetRuleExprs looks like below - // groupbyExprs: $target.name - // cacheExprs: $target.name, $target.age, $target.age + (3 * 4) - // finalCacheExprs: $target.name, $target.age + (3 * 4), $target.age - // persistExprs: $target.name, $target.age -} \ No newline at end of file diff --git a/measure/src/main/scala/org/apache/griffin/measure/rule/RuleFactory.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/RuleFactory.scala deleted file mode 100644 index d0120498f..000000000 --- a/measure/src/main/scala/org/apache/griffin/measure/rule/RuleFactory.scala +++ /dev/null @@ -1,52 +0,0 @@ -///* -//Licensed to the Apache Software Foundation (ASF) under one -//or more contributor license agreements. See the NOTICE file -//distributed with this work for additional information -//regarding copyright ownership. The ASF licenses this file -//to you under the Apache License, Version 2.0 (the -//"License"); you may not use this file except in compliance -//with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -//Unless required by applicable law or agreed to in writing, -//software distributed under the License is distributed on an -//"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -//KIND, either express or implied. See the License for the -//specific language governing permissions and limitations -//under the License. -//*/ -//package org.apache.griffin.measure.rule -// -//import org.apache.griffin.measure.config.params.user._ -// -//import scala.util.Failure -////import org.apache.griffin.measure.rule.expr_old._ -//import org.apache.griffin.measure.rule.expr._ -// -//import scala.util.{Success, Try} -// -// -//case class RuleFactory(evaluateRuleParam: EvaluateRuleParam) { -// -// val ruleParser: RuleParser = RuleParser() -// -// def generateRule(): StatementExpr = { -// val rules = evaluateRuleParam.rules -// val statement = parseExpr(rules) match { -// case Success(se) => se -// case Failure(ex) => throw ex -// } -// statement -// } -// -// private def parseExpr(rules: String): Try[StatementExpr] = { -// Try { -// val result = ruleParser.parseAll(ruleParser.rule, rules) -// if (result.successful) result.get -// else throw new Exception("parse rule error!") -//// throw new Exception("parse rule error!") -// } -// } -// -//} diff --git a/measure/src/main/scala/org/apache/griffin/measure/rule/RuleParser.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/RuleParser.scala deleted file mode 100644 index 55d9f4591..000000000 --- a/measure/src/main/scala/org/apache/griffin/measure/rule/RuleParser.scala +++ /dev/null @@ -1,244 +0,0 @@ -/* -Licensed to the Apache Software Foundation (ASF) under one -or more contributor license agreements. See the NOTICE file -distributed with this work for additional information -regarding copyright ownership. The ASF licenses this file -to you under the Apache License, Version 2.0 (the -"License"); you may not use this file except in compliance -with the License. You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, -software distributed under the License is distributed on an -"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -KIND, either express or implied. See the License for the -specific language governing permissions and limitations -under the License. -*/ -package org.apache.griffin.measure.rule - -import org.apache.griffin.measure.rule.expr._ - -import scala.util.parsing.combinator._ - -case class RuleParser() extends JavaTokenParsers with Serializable { - - /** - * BNF representation for grammar as below: - * - * ::= [WHEN ] - * rule: mapping-rule [WHEN when-rule] - * - mapping-rule: the first level opr should better not be OR | NOT, otherwise it can't automatically find the groupby column - * - when-rule: only contain the general info of data source, not the special info of each data row - * - * ::= [NOT] [(AND | OR) ]+ | "(" ")" - * logical-statement: return boolean value - * logical-operator: "AND" | "&&", "OR" | "||", "NOT" | "!" - * - * ::= ( | ) - * logical-expression example: $source.id = $target.id, $source.page_id IN ('3214', '4312', '60821') - * - * ::= "=" | "!=" | "<" | ">" | "<=" | ">=" - * ::= ["NOT"] "IN" | "BETWEEN" - * ::= "(" [] [, ]+ ")" - * range-expr example: ('3214', '4312', '60821'), (10, 15), () - * - * ::= [] [ ]+ - * math-expr example: $source.price * $target.count, "hello" + " " + "world" + 123 - * - * ::= "+" | "-" | "*" | "/" | "%" - * ::= "+" | "-" - * - * ::= | | "(" ")" - * - * ::= [ | | | ]+ - * selection example: $source.price, $source.json(), $source['state'], $source.numList[3], $target.json().mails['org' = 'apache'].names[*] - * - * ::= $source | $target - * - * ::= "." - * - * ::= "." "(" [, ]+ ")" - * ::= - * ::= - * - * ::= "[" [, ]+ "]" - * ::= | (, ) | "*" - * index-field-range: 2 means the 3rd item, (0, 3) means first 4 items, * means all items, 'age' means item 'age' - * ::= | | - * index: 0 ~ n means position from start, -1 ~ -n means position from end - * ::= ' ' | " " - * - * ::= "[" "]" - * ::= "=" | "!=" | "<" | ">" | "<=" | ">=" - * filter-sel example: ['name' = 'URL'], $source.man['age' > $source.graduate_age + 5 ] - * - * When in the selection, it mustn't contain the different , for example: - * $source.tags[1+2] valid - * $source.tags[$source.first] valid - * $source.tags[$target.first] invalid - * -- Such job is for validation, not for parser - * - * - * ::= | | | | | - * ::= - * ::= | - * ::= ("d"|"h"|"m"|"s"|"ms") - * ::= true | false - * ::= null | undefined - * ::= none - * - */ - - object Keyword { - def WhenKeywords: Parser[String] = """(?i)when""".r - def UnaryLogicalKeywords: Parser[String] = """(?i)not""".r - def BinaryLogicalKeywords: Parser[String] = """(?i)and|or""".r - def RangeKeywords: Parser[String] = """(?i)(not\s+)?(in|between)""".r - def DataSourceKeywords: Parser[String] = """(?i)\$(source|target)""".r - def Keywords: Parser[String] = WhenKeywords | UnaryLogicalKeywords | BinaryLogicalKeywords | RangeKeywords | DataSourceKeywords - } - import Keyword._ - - object Operator { - def NotLogicalOpr: Parser[String] = """(?i)not""".r | "!" - def AndLogicalOpr: Parser[String] = """(?i)and""".r | "&&" - def OrLogicalOpr: Parser[String] = """(?i)or""".r | "||" - def CompareOpr: Parser[String] = """!?==?""".r | """<=?""".r | """>=?""".r - def RangeOpr: Parser[String] = RangeKeywords - - def UnaryMathOpr: Parser[String] = "+" | "-" - def BinaryMathOpr1: Parser[String] = "*" | "/" | "%" - def BinaryMathOpr2: Parser[String] = "+" | "-" - - def FilterCompareOpr: Parser[String] = """!?==?""".r | """<=?""".r | """>=?""".r - - def SqBracketPair: (Parser[String], Parser[String]) = ("[", "]") - def BracketPair: (Parser[String], Parser[String]) = ("(", ")") - def Dot: Parser[String] = "." - def AllSelection: Parser[String] = "*" - def SQuote: Parser[String] = "'" - def DQuote: Parser[String] = "\"" - def Comma: Parser[String] = "," - } - import Operator._ - - object SomeString { -// def AnyString: Parser[String] = """[^'\"{}\[\]()=<>.$@,;+\-*/\\]*""".r - def AnyString: Parser[String] = """[^'\"]*""".r - def SimpleFieldString: Parser[String] = """\w+""".r - def FieldString: Parser[String] = """[\w\s]+""".r - def NameString: Parser[String] = """[a-zA-Z_]\w*""".r - } - import SomeString._ - - object SomeNumber { - def IntegerNumber: Parser[String] = """[+\-]?\d+""".r - def DoubleNumber: Parser[String] = """[+\-]?(\.\d+|\d+\.\d*)""".r - def IndexNumber: Parser[String] = IntegerNumber - } - import SomeNumber._ - - // -- literal -- - def literal: Parser[LiteralExpr] = literialString | literialTime | literialNumber | literialBoolean | literialNull | literialNone - def literialString: Parser[LiteralStringExpr] = (SQuote ~> AnyString <~ SQuote | DQuote ~> AnyString <~ DQuote) ^^ { LiteralStringExpr(_) } - def literialNumber: Parser[LiteralNumberExpr] = (DoubleNumber | IntegerNumber) ^^ { LiteralNumberExpr(_) } - def literialTime: Parser[LiteralTimeExpr] = """(\d+(d|h|m|s|ms))+""".r ^^ { LiteralTimeExpr(_) } - def literialBoolean: Parser[LiteralBooleanExpr] = ("""(?i)true""".r | """(?i)false""".r) ^^ { LiteralBooleanExpr(_) } - def literialNull: Parser[LiteralNullExpr] = ("""(?i)null""".r | """(?i)undefined""".r) ^^ { LiteralNullExpr(_) } - def literialNone: Parser[LiteralNoneExpr] = """(?i)none""".r ^^ { LiteralNoneExpr(_) } - - // -- selection -- - // ::= [ | | | ]+ - def selection: Parser[SelectionExpr] = selectionHead ~ rep(selector) ^^ { - case head ~ selectors => SelectionExpr(head, selectors) - } - def selector: Parser[SelectExpr] = (functionOperation | fieldSelect | indexFieldRangeSelect | filterSelect) - - def selectionHead: Parser[SelectionHead] = DataSourceKeywords ^^ { SelectionHead(_) } - // ::= "." - def fieldSelect: Parser[IndexFieldRangeSelectExpr] = Dot ~> SimpleFieldString ^^ { - case field => IndexFieldRangeSelectExpr(FieldDesc(field) :: Nil) - } - // ::= "." "(" [, ]+ ")" - def functionOperation: Parser[FunctionOperationExpr] = Dot ~ NameString ~ BracketPair._1 ~ repsep(argument, Comma) ~ BracketPair._2 ^^ { - case _ ~ func ~ _ ~ args ~ _ => FunctionOperationExpr(func, args) - } - def argument: Parser[MathExpr] = mathExpr - // ::= "[" [, ]+ "]" - def indexFieldRangeSelect: Parser[IndexFieldRangeSelectExpr] = SqBracketPair._1 ~> rep1sep(indexFieldRange, Comma) <~ SqBracketPair._2 ^^ { - case ifrs => IndexFieldRangeSelectExpr(ifrs) - } - // ::= | (, ) | "*" - def indexFieldRange: Parser[FieldDescOnly] = indexField | BracketPair._1 ~ indexField ~ Comma ~ indexField ~ BracketPair._2 ^^ { - case _ ~ if1 ~ _ ~ if2 ~ _ => FieldRangeDesc(if1, if2) - } - // ::= | | - // *here it can parse , but for simple situation, not supported now* - def indexField: Parser[FieldDescOnly] = IndexNumber ^^ { IndexDesc(_) } | fieldQuote | AllSelection ^^ { AllFieldsDesc(_) } - // ::= ' ' | " " - def fieldQuote: Parser[FieldDesc] = (SQuote ~> FieldString <~ SQuote | DQuote ~> FieldString <~ DQuote) ^^ { FieldDesc(_) } - // ::= "[" "]" - def filterSelect: Parser[FilterSelectExpr] = SqBracketPair._1 ~> fieldQuote ~ FilterCompareOpr ~ mathExpr <~ SqBracketPair._2 ^^ { - case field ~ compare ~ value => FilterSelectExpr(field, compare, value) - } - - // -- math -- - // ::= | | "(" ")" - def mathFactor: Parser[MathExpr] = (literal | selection | BracketPair._1 ~> mathExpr <~ BracketPair._2) ^^ { MathFactorExpr(_) } - // ::= [] [ ]+ - // ::= "+" | "-" - def unaryMathExpr: Parser[MathExpr] = rep(UnaryMathOpr) ~ mathFactor ^^ { - case Nil ~ a => a - case list ~ a => UnaryMathExpr(list, a) - } - // ::= "+" | "-" | "*" | "/" | "%" - def binaryMathExpr1: Parser[MathExpr] = unaryMathExpr ~ rep(BinaryMathOpr1 ~ unaryMathExpr) ^^ { - case a ~ Nil => a - case a ~ list => BinaryMathExpr(a, list.map(c => (c._1, c._2))) - } - def binaryMathExpr2: Parser[MathExpr] = binaryMathExpr1 ~ rep(BinaryMathOpr2 ~ binaryMathExpr1) ^^ { - case a ~ Nil => a - case a ~ list => BinaryMathExpr(a, list.map(c => (c._1, c._2))) - } - def mathExpr: Parser[MathExpr] = binaryMathExpr2 - - // -- logical expression -- - // ::= "(" [] [, ]+ ")" - def rangeExpr: Parser[RangeDesc] = BracketPair._1 ~> repsep(mathExpr, Comma) <~ BracketPair._2 ^^ { RangeDesc(_) } - // ::= ( | ) - def logicalExpr: Parser[LogicalExpr] = mathExpr ~ CompareOpr ~ mathExpr ^^ { - case left ~ opr ~ right => LogicalCompareExpr(left, opr, right) - } | mathExpr ~ RangeOpr ~ rangeExpr ^^ { - case left ~ opr ~ range => LogicalRangeExpr(left, opr, range) - } | mathExpr ^^ { LogicalSimpleExpr(_) } - - // -- logical statement -- - def logicalFactor: Parser[LogicalExpr] = logicalExpr | BracketPair._1 ~> logicalStatement <~ BracketPair._2 - def notLogicalStatement: Parser[LogicalExpr] = rep(NotLogicalOpr) ~ logicalFactor ^^ { - case Nil ~ a => a - case list ~ a => UnaryLogicalExpr(list, a) - } - def andLogicalStatement: Parser[LogicalExpr] = notLogicalStatement ~ rep(AndLogicalOpr ~ notLogicalStatement) ^^ { - case a ~ Nil => a - case a ~ list => BinaryLogicalExpr(a, list.map(c => (c._1, c._2))) - } - def orLogicalStatement: Parser[LogicalExpr] = andLogicalStatement ~ rep(OrLogicalOpr ~ andLogicalStatement) ^^ { - case a ~ Nil => a - case a ~ list => BinaryLogicalExpr(a, list.map(c => (c._1, c._2))) - } - // ::= [NOT] [(AND | OR) ]+ | "(" ")" - def logicalStatement: Parser[LogicalExpr] = orLogicalStatement - - // -- clause statement -- - def whereClause: Parser[WhereClauseExpr] = logicalStatement ^^ { WhereClauseExpr(_) } - def whenClause: Parser[WhenClauseExpr] = WhenKeywords ~> logicalStatement ^^ { WhenClauseExpr(_) } - - // -- rule -- - // ::= [WHEN ] - def rule: Parser[StatementExpr] = whereClause ~ opt(whenClause) ^^ { - case a ~ b => StatementExpr(a, b) - } - -} diff --git a/measure/src/main/scala/org/apache/griffin/measure/rule/SchemaValueCombineUtil.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/SchemaValueCombineUtil.scala deleted file mode 100644 index ed3b3fc21..000000000 --- a/measure/src/main/scala/org/apache/griffin/measure/rule/SchemaValueCombineUtil.scala +++ /dev/null @@ -1,187 +0,0 @@ -/* -Licensed to the Apache Software Foundation (ASF) under one -or more contributor license agreements. See the NOTICE file -distributed with this work for additional information -regarding copyright ownership. The ASF licenses this file -to you under the Apache License, Version 2.0 (the -"License"); you may not use this file except in compliance -with the License. You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, -software distributed under the License is distributed on an -"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -KIND, either express or implied. See the License for the -specific language governing permissions and limitations -under the License. -*/ -package org.apache.griffin.measure.rule - -object SchemaValueCombineUtil { - - // Map[String, List[(List[String], T)]]: Map[key, List[(path, value)]] - def cartesian[T](valuesMap: Map[String, List[(List[String], T)]]): List[Map[String, T]] = { - val fieldsList: List[(String, List[(List[String], T)])] = valuesMap.toList - - // List[key, List[(path, value)]] to List[(path, (key, value))] - val valueList: List[(List[String], (String, T))] = fieldsList.flatMap { fields => - val (key, list) = fields - list.map { pv => - val (path, value) = pv - (path, (key, value)) - } - } - - // 1. generate tree from value list, and return root node - val root = TreeUtil.genRootTree(valueList) - - // 2. deep first visit tree from root, merge datas into value map list - val valueMapList: List[Map[String, _]] = TreeUtil.mergeDatasIntoMap(root, Nil) - - // 3. simple change - val result = valueMapList.map { mp => - mp.map { kv => - val (k, v) = kv - (k, v.asInstanceOf[T]) - } - } - - result - - } - - - case class TreeNode(key: String, var datas: List[(String, _)]) { - var children = List[TreeNode]() - def addChild(node: TreeNode): Unit = children = children :+ node - def mergeSelf(node: TreeNode): Unit = datas = datas ::: node.datas - } - - object TreeUtil { - private def genTree(path: List[String], datas: List[(String, _)]): Option[TreeNode] = { - path match { - case Nil => None - case head :: tail => { - genTree(tail, datas) match { - case Some(child) => { - val curNode = TreeNode(head, Nil) - curNode.addChild(child) - Some(curNode) - } - case _ => Some(TreeNode(head, datas)) - } - } - } - } - - private def mergeTrees(trees: List[TreeNode], newTreeOpt: Option[TreeNode]): List[TreeNode] = { - newTreeOpt match { - case Some(newTree) => { - trees.find(tree => tree.key == newTree.key) match { - case Some(tree) => { - // children merge - for (child <- newTree.children) { - tree.children = mergeTrees(tree.children, Some(child)) - } - // self data merge - tree.mergeSelf(newTree) - trees - } - case _ => trees :+ newTree - } - } - case _ => trees - } - } - - private def root(): TreeNode = TreeNode("", Nil) - - def genRootTree(values: List[(List[String], (String, _))]): TreeNode = { - val rootNode = root() - val nodeOpts = values.map(value => genTree(value._1, value._2 :: Nil)) - rootNode.children = nodeOpts.foldLeft(List[TreeNode]()) { (trees, treeOpt) => - mergeTrees(trees, treeOpt) - } - rootNode - } - - private def add(mapList1: List[Map[String, _]], mapList2: List[Map[String, _]]): List[Map[String, _]] = { - mapList1 ::: mapList2 - } - private def multiply(mapList1: List[Map[String, _]], mapList2: List[Map[String, _]]): List[Map[String, _]] = { - mapList1.flatMap { map1 => - mapList2.map { map2 => - map1 ++ map2 - } - } - } - - private def keysList(mapList: List[Map[String, _]]): List[String] = { - val keySet = mapList match { - case Nil => Set[String]() - case head :: _ => head.keySet - } - keySet.toList - } - - def mergeDatasIntoMap(root: TreeNode, mapDatas: List[Map[String, _]]): List[Map[String, _]] = { - val childrenKeysMapDatas = root.children.foldLeft(Map[List[String], List[Map[String, _]]]()) { (keysMap, child) => - val childMdts = mergeDatasIntoMap(child, List[Map[String, _]]()) - childMdts match { - case Nil => keysMap - case _ => { - val keys = keysList(childMdts) - val afterList = keysMap.get(keys) match { - case Some(list) => add(list, childMdts) - case _ => childMdts - } - keysMap + (keys -> afterList) - } - } - } - val childrenMergeMaps = childrenKeysMapDatas.values.foldLeft(List[Map[String, _]]()) { (originList, list) => - originList match { - case Nil => list - case _ => multiply(originList, list) - } - } - val result = mergeNodeChildrenDatasIntoMap(root, childrenMergeMaps) - result - } - - private def mergeNodeChildrenDatasIntoMap(node: TreeNode, mapDatas: List[Map[String, _]]): List[Map[String, _]] = { - val datas: List[(String, (String, Any))] = node.children.flatMap { child => - child.datas.map(dt => (dt._1, (child.key, dt._2))) - } - val childrenDataKeys: Set[String] = datas.map(_._1).toSet - val childrenDataLists: Map[String, List[(String, _)]] = datas.foldLeft(childrenDataKeys.map(k => (k, List[(String, _)]())).toMap) { (maps, data) => - maps.get(data._1) match { - case Some(list) => maps + (data._1 -> (list :+ data._2)) - case _ => maps - } - } - - // multiply different key datas - childrenDataLists.foldLeft(mapDatas) { (mdts, klPair) => - val (key, list) = klPair - mdts match { - case Nil => list.map(pr => Map[String, Any]((key -> pr._2))) - case _ => { - list.flatMap { kvPair => - val (path, value) = kvPair - mdts.map { mp => - mp + (key -> value) - } - } - } - } - } - - } - } - - - - -} diff --git a/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/AdaptPhase.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/adaptor/AdaptPhase.scala similarity index 94% rename from measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/AdaptPhase.scala rename to measure/src/main/scala/org/apache/griffin/measure/rule/adaptor/AdaptPhase.scala index d38a4baee..26db78d44 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/AdaptPhase.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rule/adaptor/AdaptPhase.scala @@ -16,7 +16,7 @@ KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License. */ -package org.apache.griffin.measure.rules.adaptor +package org.apache.griffin.measure.rule.adaptor sealed trait AdaptPhase {} diff --git a/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/DataFrameOprAdaptor.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/adaptor/DataFrameOprAdaptor.scala similarity index 90% rename from measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/DataFrameOprAdaptor.scala rename to measure/src/main/scala/org/apache/griffin/measure/rule/adaptor/DataFrameOprAdaptor.scala index f322b947f..eb578388b 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/DataFrameOprAdaptor.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rule/adaptor/DataFrameOprAdaptor.scala @@ -16,10 +16,10 @@ KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License. */ -package org.apache.griffin.measure.rules.adaptor +package org.apache.griffin.measure.rule.adaptor -import org.apache.griffin.measure.algo.ProcessType -import org.apache.griffin.measure.rules.step._ +import org.apache.griffin.measure.process.ProcessType +import org.apache.griffin.measure.rule.step._ case class DataFrameOprAdaptor(adaptPhase: AdaptPhase) extends RuleAdaptor { diff --git a/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/GriffinDslAdaptor.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/adaptor/GriffinDslAdaptor.scala similarity index 96% rename from measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/GriffinDslAdaptor.scala rename to measure/src/main/scala/org/apache/griffin/measure/rule/adaptor/GriffinDslAdaptor.scala index f721a13b3..36cb25ab3 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/GriffinDslAdaptor.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rule/adaptor/GriffinDslAdaptor.scala @@ -16,15 +16,14 @@ KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License. */ -package org.apache.griffin.measure.rules.adaptor +package org.apache.griffin.measure.rule.adaptor -import org.apache.griffin.measure.algo.{BatchProcessType, ProcessType, StreamingProcessType} import org.apache.griffin.measure.data.connector.GroupByColumn -import org.apache.griffin.measure.rules.dsl._ -import org.apache.griffin.measure.rules.dsl.analyzer._ -import org.apache.griffin.measure.rules.dsl.expr._ -import org.apache.griffin.measure.rules.dsl.parser.GriffinDslParser -import org.apache.griffin.measure.rules.step._ +import org.apache.griffin.measure.rule.dsl._ +import org.apache.griffin.measure.rule.dsl.analyzer._ +import org.apache.griffin.measure.rule.dsl.expr._ +import org.apache.griffin.measure.rule.dsl.parser.GriffinDslParser +import org.apache.griffin.measure.rule.step._ import org.apache.griffin.measure.utils.ParamUtil._ case class GriffinDslAdaptor(dataSourceNames: Seq[String], diff --git a/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/RuleAdaptor.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/adaptor/RuleAdaptor.scala similarity index 92% rename from measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/RuleAdaptor.scala rename to measure/src/main/scala/org/apache/griffin/measure/rule/adaptor/RuleAdaptor.scala index 20f1346b1..231722581 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/RuleAdaptor.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rule/adaptor/RuleAdaptor.scala @@ -16,7 +16,7 @@ KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License. */ -package org.apache.griffin.measure.rules.adaptor +package org.apache.griffin.measure.rule.adaptor import java.util.concurrent.atomic.AtomicLong @@ -25,8 +25,8 @@ import org.apache.griffin.measure.algo._ import scala.collection.mutable.{Set => MutableSet} import org.apache.griffin.measure.config.params.user._ import org.apache.griffin.measure.log.Loggable -import org.apache.griffin.measure.rules.step.{ConcreteRuleStep, RuleStep} -import org.apache.griffin.measure.rules.dsl.{DslType, PersistType} +import org.apache.griffin.measure.rule.step.{ConcreteRuleStep, RuleStep} +import org.apache.griffin.measure.rule.dsl.{DslType, PersistType} trait RuleAdaptor extends Loggable with Serializable { diff --git a/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/RuleAdaptorGroup.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/adaptor/RuleAdaptorGroup.scala similarity index 95% rename from measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/RuleAdaptorGroup.scala rename to measure/src/main/scala/org/apache/griffin/measure/rule/adaptor/RuleAdaptorGroup.scala index d0e6511f7..237902abb 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/RuleAdaptorGroup.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rule/adaptor/RuleAdaptorGroup.scala @@ -16,13 +16,13 @@ KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License. */ -package org.apache.griffin.measure.rules.adaptor +package org.apache.griffin.measure.rule.adaptor -import org.apache.griffin.measure.algo.ProcessType import org.apache.griffin.measure.config.params.user._ +import org.apache.griffin.measure.process.ProcessType import org.apache.griffin.measure.process.check.DataChecker -import org.apache.griffin.measure.rules.dsl._ -import org.apache.griffin.measure.rules.step._ +import org.apache.griffin.measure.rule.dsl._ +import org.apache.griffin.measure.rule.step._ import org.apache.spark.sql.SQLContext import scala.collection.mutable.{Map => MutableMap} diff --git a/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/SparkSqlAdaptor.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/adaptor/SparkSqlAdaptor.scala similarity index 95% rename from measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/SparkSqlAdaptor.scala rename to measure/src/main/scala/org/apache/griffin/measure/rule/adaptor/SparkSqlAdaptor.scala index 643ef2d9b..7d5a24cae 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rules/adaptor/SparkSqlAdaptor.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rule/adaptor/SparkSqlAdaptor.scala @@ -16,11 +16,11 @@ KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License. */ -package org.apache.griffin.measure.rules.adaptor +package org.apache.griffin.measure.rule.adaptor import org.apache.griffin.measure.algo._ import org.apache.griffin.measure.data.connector.GroupByColumn -import org.apache.griffin.measure.rules.step._ +import org.apache.griffin.measure.rule.step._ case class SparkSqlAdaptor(adaptPhase: AdaptPhase) extends RuleAdaptor { diff --git a/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/DqType.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/dsl/DqType.scala similarity index 97% rename from measure/src/main/scala/org/apache/griffin/measure/rules/dsl/DqType.scala rename to measure/src/main/scala/org/apache/griffin/measure/rule/dsl/DqType.scala index b250cb9fd..ac2740391 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/DqType.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rule/dsl/DqType.scala @@ -16,7 +16,7 @@ KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License. */ -package org.apache.griffin.measure.rules.dsl +package org.apache.griffin.measure.rule.dsl import scala.util.matching.Regex diff --git a/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/DslType.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/dsl/DslType.scala similarity index 97% rename from measure/src/main/scala/org/apache/griffin/measure/rules/dsl/DslType.scala rename to measure/src/main/scala/org/apache/griffin/measure/rule/dsl/DslType.scala index 8e9e8edb2..cfda393e2 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/DslType.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rule/dsl/DslType.scala @@ -16,7 +16,7 @@ KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License. */ -package org.apache.griffin.measure.rules.dsl +package org.apache.griffin.measure.rule.dsl import scala.util.matching.Regex diff --git a/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/PersistType.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/dsl/PersistType.scala similarity index 97% rename from measure/src/main/scala/org/apache/griffin/measure/rules/dsl/PersistType.scala rename to measure/src/main/scala/org/apache/griffin/measure/rule/dsl/PersistType.scala index b8f3c4653..10b83c884 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/PersistType.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rule/dsl/PersistType.scala @@ -16,7 +16,7 @@ KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License. */ -package org.apache.griffin.measure.rules.dsl +package org.apache.griffin.measure.rule.dsl import scala.util.matching.Regex diff --git a/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/analyzer/AccuracyAnalyzer.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/dsl/analyzer/AccuracyAnalyzer.scala similarity index 93% rename from measure/src/main/scala/org/apache/griffin/measure/rules/dsl/analyzer/AccuracyAnalyzer.scala rename to measure/src/main/scala/org/apache/griffin/measure/rule/dsl/analyzer/AccuracyAnalyzer.scala index 1071a967b..7efb32ee0 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/analyzer/AccuracyAnalyzer.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rule/dsl/analyzer/AccuracyAnalyzer.scala @@ -16,9 +16,9 @@ KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License. */ -package org.apache.griffin.measure.rules.dsl.analyzer +package org.apache.griffin.measure.rule.dsl.analyzer -import org.apache.griffin.measure.rules.dsl.expr._ +import org.apache.griffin.measure.rule.dsl.expr._ case class AccuracyAnalyzer(expr: LogicalExpr, sourceName: String, targetName: String) extends BasicAnalyzer { diff --git a/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/analyzer/BasicAnalyzer.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/dsl/analyzer/BasicAnalyzer.scala similarity index 94% rename from measure/src/main/scala/org/apache/griffin/measure/rules/dsl/analyzer/BasicAnalyzer.scala rename to measure/src/main/scala/org/apache/griffin/measure/rule/dsl/analyzer/BasicAnalyzer.scala index 94516ce35..063eb7b45 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/analyzer/BasicAnalyzer.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rule/dsl/analyzer/BasicAnalyzer.scala @@ -16,9 +16,9 @@ KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License. */ -package org.apache.griffin.measure.rules.dsl.analyzer +package org.apache.griffin.measure.rule.dsl.analyzer -import org.apache.griffin.measure.rules.dsl.expr._ +import org.apache.griffin.measure.rule.dsl.expr._ trait BasicAnalyzer extends Serializable { diff --git a/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/analyzer/ProfilingAnalyzer.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/dsl/analyzer/ProfilingAnalyzer.scala similarity index 92% rename from measure/src/main/scala/org/apache/griffin/measure/rules/dsl/analyzer/ProfilingAnalyzer.scala rename to measure/src/main/scala/org/apache/griffin/measure/rule/dsl/analyzer/ProfilingAnalyzer.scala index 6424eb3d7..47b77037a 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/analyzer/ProfilingAnalyzer.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rule/dsl/analyzer/ProfilingAnalyzer.scala @@ -16,9 +16,9 @@ KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License. */ -package org.apache.griffin.measure.rules.dsl.analyzer +package org.apache.griffin.measure.rule.dsl.analyzer -import org.apache.griffin.measure.rules.dsl.expr._ +import org.apache.griffin.measure.rule.dsl.expr._ case class ProfilingAnalyzer(expr: CombinedClause, sourceName: String) extends BasicAnalyzer { diff --git a/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/expr/AliasableExpr.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/dsl/expr/AliasableExpr.scala similarity index 94% rename from measure/src/main/scala/org/apache/griffin/measure/rules/dsl/expr/AliasableExpr.scala rename to measure/src/main/scala/org/apache/griffin/measure/rule/dsl/expr/AliasableExpr.scala index b6ad00a7c..33a12e00b 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/expr/AliasableExpr.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rule/dsl/expr/AliasableExpr.scala @@ -16,7 +16,7 @@ KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License. */ -package org.apache.griffin.measure.rules.dsl.expr +package org.apache.griffin.measure.rule.dsl.expr trait AliasableExpr extends Expr { diff --git a/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/expr/ClauseExpression.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/dsl/expr/ClauseExpression.scala similarity index 98% rename from measure/src/main/scala/org/apache/griffin/measure/rules/dsl/expr/ClauseExpression.scala rename to measure/src/main/scala/org/apache/griffin/measure/rule/dsl/expr/ClauseExpression.scala index 0bf365dfd..e7bd84f1d 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/expr/ClauseExpression.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rule/dsl/expr/ClauseExpression.scala @@ -16,7 +16,7 @@ KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License. */ -package org.apache.griffin.measure.rules.dsl.expr +package org.apache.griffin.measure.rule.dsl.expr trait ClauseExpression extends Expr { } diff --git a/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/expr/Expr.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/dsl/expr/Expr.scala similarity index 94% rename from measure/src/main/scala/org/apache/griffin/measure/rules/dsl/expr/Expr.scala rename to measure/src/main/scala/org/apache/griffin/measure/rule/dsl/expr/Expr.scala index 64b02ff6b..850579c3f 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/expr/Expr.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rule/dsl/expr/Expr.scala @@ -16,7 +16,7 @@ KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License. */ -package org.apache.griffin.measure.rules.dsl.expr +package org.apache.griffin.measure.rule.dsl.expr trait Expr extends TreeNode with Serializable { diff --git a/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/expr/FunctionExpr.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/dsl/expr/FunctionExpr.scala similarity index 95% rename from measure/src/main/scala/org/apache/griffin/measure/rules/dsl/expr/FunctionExpr.scala rename to measure/src/main/scala/org/apache/griffin/measure/rule/dsl/expr/FunctionExpr.scala index 910e36452..b82fd96b1 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/expr/FunctionExpr.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rule/dsl/expr/FunctionExpr.scala @@ -16,7 +16,7 @@ KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License. */ -package org.apache.griffin.measure.rules.dsl.expr +package org.apache.griffin.measure.rule.dsl.expr case class FunctionExpr(functionName: String, args: Seq[Expr], aliasOpt: Option[String] ) extends Expr with AliasableExpr { diff --git a/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/expr/LiteralExpr.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/dsl/expr/LiteralExpr.scala similarity index 97% rename from measure/src/main/scala/org/apache/griffin/measure/rules/dsl/expr/LiteralExpr.scala rename to measure/src/main/scala/org/apache/griffin/measure/rule/dsl/expr/LiteralExpr.scala index 9b0cc29a3..60290bce3 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/expr/LiteralExpr.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rule/dsl/expr/LiteralExpr.scala @@ -16,7 +16,7 @@ KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License. */ -package org.apache.griffin.measure.rules.dsl.expr +package org.apache.griffin.measure.rule.dsl.expr import org.apache.griffin.measure.utils.TimeUtil diff --git a/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/expr/LogicalExpr.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/dsl/expr/LogicalExpr.scala similarity index 98% rename from measure/src/main/scala/org/apache/griffin/measure/rules/dsl/expr/LogicalExpr.scala rename to measure/src/main/scala/org/apache/griffin/measure/rule/dsl/expr/LogicalExpr.scala index 824db19c7..8e28b6763 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/expr/LogicalExpr.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rule/dsl/expr/LogicalExpr.scala @@ -16,7 +16,7 @@ KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License. */ -package org.apache.griffin.measure.rules.dsl.expr +package org.apache.griffin.measure.rule.dsl.expr trait LogicalExpr extends Expr { } diff --git a/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/expr/MathExpr.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/dsl/expr/MathExpr.scala similarity index 97% rename from measure/src/main/scala/org/apache/griffin/measure/rules/dsl/expr/MathExpr.scala rename to measure/src/main/scala/org/apache/griffin/measure/rule/dsl/expr/MathExpr.scala index 9d4e54d64..b3d3db430 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/expr/MathExpr.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rule/dsl/expr/MathExpr.scala @@ -16,7 +16,7 @@ KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License. */ -package org.apache.griffin.measure.rules.dsl.expr +package org.apache.griffin.measure.rule.dsl.expr trait MathExpr extends Expr { } diff --git a/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/expr/SelectExpr.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/dsl/expr/SelectExpr.scala similarity index 98% rename from measure/src/main/scala/org/apache/griffin/measure/rules/dsl/expr/SelectExpr.scala rename to measure/src/main/scala/org/apache/griffin/measure/rule/dsl/expr/SelectExpr.scala index f4fc92d05..039d21591 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/expr/SelectExpr.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rule/dsl/expr/SelectExpr.scala @@ -16,7 +16,7 @@ KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License. */ -package org.apache.griffin.measure.rules.dsl.expr +package org.apache.griffin.measure.rule.dsl.expr trait HeadExpr extends Expr { diff --git a/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/expr/TreeNode.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/dsl/expr/TreeNode.scala similarity index 97% rename from measure/src/main/scala/org/apache/griffin/measure/rules/dsl/expr/TreeNode.scala rename to measure/src/main/scala/org/apache/griffin/measure/rule/dsl/expr/TreeNode.scala index d9bb85a13..aab16b401 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/expr/TreeNode.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rule/dsl/expr/TreeNode.scala @@ -16,7 +16,7 @@ KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License. */ -package org.apache.griffin.measure.rules.dsl.expr +package org.apache.griffin.measure.rule.dsl.expr trait TreeNode extends Serializable { diff --git a/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/parser/BasicParser.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/dsl/parser/BasicParser.scala similarity index 99% rename from measure/src/main/scala/org/apache/griffin/measure/rules/dsl/parser/BasicParser.scala rename to measure/src/main/scala/org/apache/griffin/measure/rule/dsl/parser/BasicParser.scala index 2a022cf72..7d13e03b4 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/parser/BasicParser.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rule/dsl/parser/BasicParser.scala @@ -16,9 +16,9 @@ KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License. */ -package org.apache.griffin.measure.rules.dsl.parser +package org.apache.griffin.measure.rule.dsl.parser -import org.apache.griffin.measure.rules.dsl.expr._ +import org.apache.griffin.measure.rule.dsl.expr._ import scala.util.parsing.combinator.JavaTokenParsers diff --git a/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/parser/GriffinDslParser.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/dsl/parser/GriffinDslParser.scala similarity index 88% rename from measure/src/main/scala/org/apache/griffin/measure/rules/dsl/parser/GriffinDslParser.scala rename to measure/src/main/scala/org/apache/griffin/measure/rule/dsl/parser/GriffinDslParser.scala index b3959caf3..caa995659 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rules/dsl/parser/GriffinDslParser.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rule/dsl/parser/GriffinDslParser.scala @@ -16,10 +16,10 @@ KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License. */ -package org.apache.griffin.measure.rules.dsl.parser +package org.apache.griffin.measure.rule.dsl.parser -import org.apache.griffin.measure.rules.dsl._ -import org.apache.griffin.measure.rules.dsl.expr.Expr +import org.apache.griffin.measure.rule.dsl._ +import org.apache.griffin.measure.rule.dsl.expr.Expr case class GriffinDslParser(dataSourceNames: Seq[String], functionNames: Seq[String] ) extends BasicParser { diff --git a/measure/src/main/scala/org/apache/griffin/measure/rule/expr/AnalyzableExpr.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/expr/AnalyzableExpr.scala deleted file mode 100644 index aefcaaddf..000000000 --- a/measure/src/main/scala/org/apache/griffin/measure/rule/expr/AnalyzableExpr.scala +++ /dev/null @@ -1,24 +0,0 @@ -/* -Licensed to the Apache Software Foundation (ASF) under one -or more contributor license agreements. See the NOTICE file -distributed with this work for additional information -regarding copyright ownership. The ASF licenses this file -to you under the Apache License, Version 2.0 (the -"License"); you may not use this file except in compliance -with the License. You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, -software distributed under the License is distributed on an -"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -KIND, either express or implied. See the License for the -specific language governing permissions and limitations -under the License. -*/ -package org.apache.griffin.measure.rule.expr - - -trait AnalyzableExpr extends Serializable { - def getGroupbyExprPairs(dsPair: (String, String)): Seq[(Expr, Expr)] = Nil -} \ No newline at end of file diff --git a/measure/src/main/scala/org/apache/griffin/measure/rule/expr/Cacheable.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/expr/Cacheable.scala deleted file mode 100644 index feb81569b..000000000 --- a/measure/src/main/scala/org/apache/griffin/measure/rule/expr/Cacheable.scala +++ /dev/null @@ -1,33 +0,0 @@ -/* -Licensed to the Apache Software Foundation (ASF) under one -or more contributor license agreements. See the NOTICE file -distributed with this work for additional information -regarding copyright ownership. The ASF licenses this file -to you under the Apache License, Version 2.0 (the -"License"); you may not use this file except in compliance -with the License. You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, -software distributed under the License is distributed on an -"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -KIND, either express or implied. See the License for the -specific language governing permissions and limitations -under the License. -*/ -package org.apache.griffin.measure.rule.expr - -trait Cacheable extends DataSourceable { - protected def cacheUnit: Boolean = false - def cacheable(ds: String): Boolean = { - cacheUnit && !conflict() && ((ds.isEmpty && dataSources.isEmpty) || (ds.nonEmpty && contains(ds))) - } - protected def getCacheExprs(ds: String): Iterable[Cacheable] - - protected def persistUnit: Boolean = false - def persistable(ds: String): Boolean = { - persistUnit && ((ds.isEmpty && dataSources.isEmpty) || (ds.nonEmpty && contains(ds))) - } - protected def getPersistExprs(ds: String): Iterable[Cacheable] -} diff --git a/measure/src/main/scala/org/apache/griffin/measure/rule/expr/Calculatable.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/expr/Calculatable.scala deleted file mode 100644 index 904e823e4..000000000 --- a/measure/src/main/scala/org/apache/griffin/measure/rule/expr/Calculatable.scala +++ /dev/null @@ -1,25 +0,0 @@ -/* -Licensed to the Apache Software Foundation (ASF) under one -or more contributor license agreements. See the NOTICE file -distributed with this work for additional information -regarding copyright ownership. The ASF licenses this file -to you under the Apache License, Version 2.0 (the -"License"); you may not use this file except in compliance -with the License. You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, -software distributed under the License is distributed on an -"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -KIND, either express or implied. See the License for the -specific language governing permissions and limitations -under the License. -*/ -package org.apache.griffin.measure.rule.expr - -trait Calculatable extends Serializable { - - def calculate(values: Map[String, Any]): Option[Any] - -} diff --git a/measure/src/main/scala/org/apache/griffin/measure/rule/expr/ClauseExpr.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/expr/ClauseExpr.scala deleted file mode 100644 index a56e0db94..000000000 --- a/measure/src/main/scala/org/apache/griffin/measure/rule/expr/ClauseExpr.scala +++ /dev/null @@ -1,109 +0,0 @@ -/* -Licensed to the Apache Software Foundation (ASF) under one -or more contributor license agreements. See the NOTICE file -distributed with this work for additional information -regarding copyright ownership. The ASF licenses this file -to you under the Apache License, Version 2.0 (the -"License"); you may not use this file except in compliance -with the License. You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, -software distributed under the License is distributed on an -"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -KIND, either express or implied. See the License for the -specific language governing permissions and limitations -under the License. -*/ -package org.apache.griffin.measure.rule.expr - - -trait ClauseExpr extends Expr with AnalyzableExpr { - def valid(values: Map[String, Any]): Boolean = true - override def cacheUnit: Boolean = true -} - -case class WhereClauseExpr(expr: LogicalExpr) extends ClauseExpr { - def calculateOnly(values: Map[String, Any]): Option[Any] = expr.calculate(values) - val desc: String = expr.desc - val dataSources: Set[String] = expr.dataSources - override def getSubCacheExprs(ds: String): Iterable[Expr] = { - expr.getCacheExprs(ds) - } - override def getSubFinalCacheExprs(ds: String): Iterable[Expr] = { - expr.getFinalCacheExprs(ds) - } - override def getSubPersistExprs(ds: String): Iterable[Expr] = { - expr.getPersistExprs(ds) - } - - override def getGroupbyExprPairs(dsPair: (String, String)): Seq[(Expr, Expr)] = expr.getGroupbyExprPairs(dsPair) -} - -case class WhenClauseExpr(expr: LogicalExpr) extends ClauseExpr { - def calculateOnly(values: Map[String, Any]): Option[Any] = expr.calculate(values) - val desc: String = s"WHEN ${expr.desc}" - val dataSources: Set[String] = expr.dataSources - override def getSubCacheExprs(ds: String): Iterable[Expr] = { - expr.getCacheExprs(ds) - } - override def getSubFinalCacheExprs(ds: String): Iterable[Expr] = { - expr.getFinalCacheExprs(ds) - } - override def getSubPersistExprs(ds: String): Iterable[Expr] = { - expr.getPersistExprs(ds) - } - - override def getGroupbyExprPairs(dsPair: (String, String)): Seq[(Expr, Expr)] = expr.getGroupbyExprPairs(dsPair) -} - -case class StatementExpr(whereClause: WhereClauseExpr, whenClauseOpt: Option[WhenClauseExpr]) extends ClauseExpr { - def calculateOnly(values: Map[String, Any]): Option[Any] = whereClause.calculate(values) - val desc: String = { - whenClauseOpt match { - case Some(expr) => s"${whereClause.desc} ${expr.desc}" - case _ => whereClause.desc - } - } - val dataSources: Set[String] = whereClause.dataSources - override def getSubCacheExprs(ds: String): Iterable[Expr] = { - whereClause.getCacheExprs(ds) - } - override def getSubFinalCacheExprs(ds: String): Iterable[Expr] = { - whereClause.getFinalCacheExprs(ds) - } - override def getSubPersistExprs(ds: String): Iterable[Expr] = { - whereClause.getPersistExprs(ds) - } - - override def getGroupbyExprPairs(dsPair: (String, String)): Seq[(Expr, Expr)] = whereClause.getGroupbyExprPairs(dsPair) -} - -//case class WhenClauseStatementExpr(expr: LogicalExpr, whenExpr: LogicalExpr) extends ClauseExpr { -// def calculateOnly(values: Map[String, Any]): Option[Any] = expr.calculate(values) -// val desc: String = s"${expr.desc} when ${whenExpr.desc}" -// -// override def valid(values: Map[String, Any]): Boolean = { -// whenExpr.calculate(values) match { -// case Some(r: Boolean) => r -// case _ => false -// } -// } -// -// val dataSources: Set[String] = expr.dataSources ++ whenExpr.dataSources -// override def getSubCacheExprs(ds: String): Iterable[Expr] = { -// expr.getCacheExprs(ds) ++ whenExpr.getCacheExprs(ds) -// } -// override def getSubFinalCacheExprs(ds: String): Iterable[Expr] = { -// expr.getFinalCacheExprs(ds) ++ whenExpr.getFinalCacheExprs(ds) -// } -// override def getSubPersistExprs(ds: String): Iterable[Expr] = { -// expr.getPersistExprs(ds) ++ whenExpr.getPersistExprs(ds) -// } -// -// override def getGroupbyExprPairs(dsPair: (String, String)): Seq[(Expr, Expr)] = { -// expr.getGroupbyExprPairs(dsPair) ++ whenExpr.getGroupbyExprPairs(dsPair) -// } -// override def getWhenClauseExpr(): Option[LogicalExpr] = Some(whenExpr) -//} \ No newline at end of file diff --git a/measure/src/main/scala/org/apache/griffin/measure/rule/expr/DataSourceable.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/expr/DataSourceable.scala deleted file mode 100644 index e2cf17240..000000000 --- a/measure/src/main/scala/org/apache/griffin/measure/rule/expr/DataSourceable.scala +++ /dev/null @@ -1,28 +0,0 @@ -/* -Licensed to the Apache Software Foundation (ASF) under one -or more contributor license agreements. See the NOTICE file -distributed with this work for additional information -regarding copyright ownership. The ASF licenses this file -to you under the Apache License, Version 2.0 (the -"License"); you may not use this file except in compliance -with the License. You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, -software distributed under the License is distributed on an -"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -KIND, either express or implied. See the License for the -specific language governing permissions and limitations -under the License. -*/ -package org.apache.griffin.measure.rule.expr - -trait DataSourceable extends Serializable { - val dataSources: Set[String] - protected def conflict(): Boolean = dataSources.size > 1 - def contains(ds: String): Boolean = dataSources.contains(ds) - def dataSourceOpt: Option[String] = { - if (dataSources.size == 1) Some(dataSources.head) else None - } -} \ No newline at end of file diff --git a/measure/src/main/scala/org/apache/griffin/measure/rule/expr/Describable.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/expr/Describable.scala deleted file mode 100644 index 393d7a6d3..000000000 --- a/measure/src/main/scala/org/apache/griffin/measure/rule/expr/Describable.scala +++ /dev/null @@ -1,33 +0,0 @@ -/* -Licensed to the Apache Software Foundation (ASF) under one -or more contributor license agreements. See the NOTICE file -distributed with this work for additional information -regarding copyright ownership. The ASF licenses this file -to you under the Apache License, Version 2.0 (the -"License"); you may not use this file except in compliance -with the License. You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, -software distributed under the License is distributed on an -"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -KIND, either express or implied. See the License for the -specific language governing permissions and limitations -under the License. -*/ -package org.apache.griffin.measure.rule.expr - -trait Describable extends Serializable { - - val desc: String - - protected def describe(v: Any): String = { - v match { - case s: Describable => s"${s.desc}" - case s: String => s"'${s}'" - case a => s"${a}" - } - } - -} diff --git a/measure/src/main/scala/org/apache/griffin/measure/rule/expr/Expr.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/expr/Expr.scala deleted file mode 100644 index 726b5b655..000000000 --- a/measure/src/main/scala/org/apache/griffin/measure/rule/expr/Expr.scala +++ /dev/null @@ -1,53 +0,0 @@ -/* -Licensed to the Apache Software Foundation (ASF) under one -or more contributor license agreements. See the NOTICE file -distributed with this work for additional information -regarding copyright ownership. The ASF licenses this file -to you under the Apache License, Version 2.0 (the -"License"); you may not use this file except in compliance -with the License. You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, -software distributed under the License is distributed on an -"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -KIND, either express or implied. See the License for the -specific language governing permissions and limitations -under the License. -*/ -package org.apache.griffin.measure.rule.expr - -import org.apache.spark.sql.types.DataType - -trait Expr extends Serializable with Describable with Cacheable with Calculatable { - - protected val _defaultId: String = ExprIdCounter.emptyId - - val _id = ExprIdCounter.genId(_defaultId) - - protected def getSubCacheExprs(ds: String): Iterable[Expr] = Nil - final def getCacheExprs(ds: String): Iterable[Expr] = { - if (cacheable(ds)) getSubCacheExprs(ds).toList :+ this else getSubCacheExprs(ds) - } - - protected def getSubFinalCacheExprs(ds: String): Iterable[Expr] = Nil - final def getFinalCacheExprs(ds: String): Iterable[Expr] = { - if (cacheable(ds)) Nil :+ this else getSubFinalCacheExprs(ds) - } - - protected def getSubPersistExprs(ds: String): Iterable[Expr] = Nil - final def getPersistExprs(ds: String): Iterable[Expr] = { - if (persistable(ds)) getSubPersistExprs(ds).toList :+ this else getSubPersistExprs(ds) - } - - final def calculate(values: Map[String, Any]): Option[Any] = { - values.get(_id) match { - case Some(v) => Some(v) - case _ => calculateOnly(values) - } - } - protected def calculateOnly(values: Map[String, Any]): Option[Any] - -} - diff --git a/measure/src/main/scala/org/apache/griffin/measure/rule/expr/ExprDescOnly.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/expr/ExprDescOnly.scala deleted file mode 100644 index 01b7e3cdf..000000000 --- a/measure/src/main/scala/org/apache/griffin/measure/rule/expr/ExprDescOnly.scala +++ /dev/null @@ -1,40 +0,0 @@ -/* -Licensed to the Apache Software Foundation (ASF) under one -or more contributor license agreements. See the NOTICE file -distributed with this work for additional information -regarding copyright ownership. The ASF licenses this file -to you under the Apache License, Version 2.0 (the -"License"); you may not use this file except in compliance -with the License. You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, -software distributed under the License is distributed on an -"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -KIND, either express or implied. See the License for the -specific language governing permissions and limitations -under the License. -*/ -package org.apache.griffin.measure.rule.expr - -trait ExprDescOnly extends Describable { - -} - - -case class SelectionHead(expr: String) extends ExprDescOnly { - private val headRegex = """\$(\w+)""".r - val head: String = expr match { - case headRegex(v) => v.toLowerCase - case _ => expr - } - val desc: String = "$" + head -} - -case class RangeDesc(elements: Iterable[MathExpr]) extends ExprDescOnly { - val desc: String = { - val rangeDesc = elements.map(_.desc).mkString(", ") - s"(${rangeDesc})" - } -} diff --git a/measure/src/main/scala/org/apache/griffin/measure/rule/expr/ExprIdCounter.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/expr/ExprIdCounter.scala deleted file mode 100644 index ae76aefb6..000000000 --- a/measure/src/main/scala/org/apache/griffin/measure/rule/expr/ExprIdCounter.scala +++ /dev/null @@ -1,60 +0,0 @@ -/* -Licensed to the Apache Software Foundation (ASF) under one -or more contributor license agreements. See the NOTICE file -distributed with this work for additional information -regarding copyright ownership. The ASF licenses this file -to you under the Apache License, Version 2.0 (the -"License"); you may not use this file except in compliance -with the License. You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, -software distributed under the License is distributed on an -"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -KIND, either express or implied. See the License for the -specific language governing permissions and limitations -under the License. -*/ -package org.apache.griffin.measure.rule.expr - -import java.util.concurrent.atomic.AtomicLong - -import scala.collection.mutable.{Set => MutableSet} - -object ExprIdCounter { - - private val idCounter: AtomicLong = new AtomicLong(0L) - - private val existIdSet: MutableSet[String] = MutableSet.empty[String] - - private val invalidIdRegex = """^\d+$""".r - - val emptyId: String = "" - - def genId(defaultId: String): String = { - defaultId match { - case emptyId => increment.toString - case invalidIdRegex() => increment.toString -// case defId if (exist(defId)) => s"${increment}#${defId}" - case defId if (exist(defId)) => s"${defId}" - case _ => { - insertUserId(defaultId) - defaultId - } - } - } - - private def exist(id: String): Boolean = { - existIdSet.contains(id) - } - - private def insertUserId(id: String): Unit = { - existIdSet += id - } - - private def increment(): Long = { - idCounter.incrementAndGet() - } - -} diff --git a/measure/src/main/scala/org/apache/griffin/measure/rule/expr/FieldDescOnly.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/expr/FieldDescOnly.scala deleted file mode 100644 index dca037b81..000000000 --- a/measure/src/main/scala/org/apache/griffin/measure/rule/expr/FieldDescOnly.scala +++ /dev/null @@ -1,58 +0,0 @@ -/* -Licensed to the Apache Software Foundation (ASF) under one -or more contributor license agreements. See the NOTICE file -distributed with this work for additional information -regarding copyright ownership. The ASF licenses this file -to you under the Apache License, Version 2.0 (the -"License"); you may not use this file except in compliance -with the License. You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, -software distributed under the License is distributed on an -"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -KIND, either express or implied. See the License for the -specific language governing permissions and limitations -under the License. -*/ -package org.apache.griffin.measure.rule.expr - -import scala.util.{Success, Try} - -trait FieldDescOnly extends Describable with DataSourceable { - -} - -case class IndexDesc(expr: String) extends FieldDescOnly { - val index: Int = { - Try(expr.toInt) match { - case Success(v) => v - case _ => throw new Exception(s"${expr} is invalid index") - } - } - val desc: String = describe(index) - val dataSources: Set[String] = Set.empty[String] -} - -case class FieldDesc(expr: String) extends FieldDescOnly { - val field: String = expr - val desc: String = describe(field) - val dataSources: Set[String] = Set.empty[String] -} - -case class AllFieldsDesc(expr: String) extends FieldDescOnly { - val allFields: String = expr - val desc: String = allFields - val dataSources: Set[String] = Set.empty[String] -} - -case class FieldRangeDesc(startField: FieldDescOnly, endField: FieldDescOnly) extends FieldDescOnly { - val desc: String = { - (startField, endField) match { - case (f1: IndexDesc, f2: IndexDesc) => s"(${f1.desc}, ${f2.desc})" - case _ => throw new Exception("invalid field range description") - } - } - val dataSources: Set[String] = Set.empty[String] -} \ No newline at end of file diff --git a/measure/src/main/scala/org/apache/griffin/measure/rule/expr/LiteralExpr.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/expr/LiteralExpr.scala deleted file mode 100644 index acf158918..000000000 --- a/measure/src/main/scala/org/apache/griffin/measure/rule/expr/LiteralExpr.scala +++ /dev/null @@ -1,83 +0,0 @@ -/* -Licensed to the Apache Software Foundation (ASF) under one -or more contributor license agreements. See the NOTICE file -distributed with this work for additional information -regarding copyright ownership. The ASF licenses this file -to you under the Apache License, Version 2.0 (the -"License"); you may not use this file except in compliance -with the License. You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, -software distributed under the License is distributed on an -"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -KIND, either express or implied. See the License for the -specific language governing permissions and limitations -under the License. -*/ -package org.apache.griffin.measure.rule.expr - -import org.apache.griffin.measure.utils.TimeUtil -import org.apache.spark.sql.types._ - -import scala.util.{Failure, Success, Try} - -trait LiteralExpr extends Expr { - val value: Option[Any] - def calculateOnly(values: Map[String, Any]): Option[Any] = value - val dataSources: Set[String] = Set.empty[String] -} - -case class LiteralValueExpr(value: Option[Any]) extends LiteralExpr { - val desc: String = value.getOrElse("").toString -} - -case class LiteralStringExpr(expr: String) extends LiteralExpr { - val value: Option[String] = Some(expr) - val desc: String = s"'${value.getOrElse("")}'" -} - -case class LiteralNumberExpr(expr: String) extends LiteralExpr { - val value: Option[Any] = { - if (expr.contains(".")) { - Try (expr.toDouble) match { - case Success(v) => Some(v) - case _ => throw new Exception(s"${expr} is invalid number") - } - } else { - Try (expr.toLong) match { - case Success(v) => Some(v) - case _ => throw new Exception(s"${expr} is invalid number") - } - } - } - val desc: String = value.getOrElse("").toString -} - -case class LiteralTimeExpr(expr: String) extends LiteralExpr { - final val TimeRegex = """(\d+)(d|h|m|s|ms)""".r - val value: Option[Long] = TimeUtil.milliseconds(expr) - val desc: String = expr -} - -case class LiteralBooleanExpr(expr: String) extends LiteralExpr { - final val TrueRegex = """(?i)true""".r - final val FalseRegex = """(?i)false""".r - val value: Option[Boolean] = expr match { - case TrueRegex() => Some(true) - case FalseRegex() => Some(false) - case _ => throw new Exception(s"${expr} is invalid boolean") - } - val desc: String = value.getOrElse("").toString -} - -case class LiteralNullExpr(expr: String) extends LiteralExpr { - val value: Option[Any] = Some(null) - val desc: String = "null" -} - -case class LiteralNoneExpr(expr: String) extends LiteralExpr { - val value: Option[Any] = None - val desc: String = "none" -} \ No newline at end of file diff --git a/measure/src/main/scala/org/apache/griffin/measure/rule/expr/LogicalExpr.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/expr/LogicalExpr.scala deleted file mode 100644 index dd061d723..000000000 --- a/measure/src/main/scala/org/apache/griffin/measure/rule/expr/LogicalExpr.scala +++ /dev/null @@ -1,178 +0,0 @@ -/* -Licensed to the Apache Software Foundation (ASF) under one -or more contributor license agreements. See the NOTICE file -distributed with this work for additional information -regarding copyright ownership. The ASF licenses this file -to you under the Apache License, Version 2.0 (the -"License"); you may not use this file except in compliance -with the License. You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, -software distributed under the License is distributed on an -"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -KIND, either express or implied. See the License for the -specific language governing permissions and limitations -under the License. -*/ -package org.apache.griffin.measure.rule.expr - -import org.apache.griffin.measure.rule.CalculationUtil._ -import org.apache.spark.sql.types.{BooleanType, DataType} - -trait LogicalExpr extends Expr with AnalyzableExpr { - override def cacheUnit: Boolean = true -} - -case class LogicalSimpleExpr(expr: MathExpr) extends LogicalExpr { - def calculateOnly(values: Map[String, Any]): Option[Any] = expr.calculate(values) - val desc: String = expr.desc - val dataSources: Set[String] = expr.dataSources - override def cacheUnit: Boolean = false - override def getSubCacheExprs(ds: String): Iterable[Expr] = expr.getCacheExprs(ds) - override def getSubFinalCacheExprs(ds: String): Iterable[Expr] = expr.getFinalCacheExprs(ds) - override def getSubPersistExprs(ds: String): Iterable[Expr] = expr.getPersistExprs(ds) -} - -case class LogicalCompareExpr(left: MathExpr, compare: String, right: MathExpr) extends LogicalExpr { - private val (eqOpr, neqOpr, btOpr, bteOpr, ltOpr, lteOpr) = ("""==?""".r, """!==?""".r, ">", ">=", "<", "<=") - def calculateOnly(values: Map[String, Any]): Option[Boolean] = { - val (lv, rv) = (left.calculate(values), right.calculate(values)) - compare match { - case this.eqOpr() => lv === rv - case this.neqOpr() => lv =!= rv - case this.btOpr => lv > rv - case this.bteOpr => lv >= rv - case this.ltOpr => lv < rv - case this.lteOpr => lv <= rv - case _ => None - } - } - val desc: String = s"${left.desc} ${compare} ${right.desc}" - val dataSources: Set[String] = left.dataSources ++ right.dataSources - override def getSubCacheExprs(ds: String): Iterable[Expr] = { - left.getCacheExprs(ds) ++ right.getCacheExprs(ds) - } - override def getSubFinalCacheExprs(ds: String): Iterable[Expr] = { - left.getFinalCacheExprs(ds) ++ right.getFinalCacheExprs(ds) - } - override def getSubPersistExprs(ds: String): Iterable[Expr] = { - left.getPersistExprs(ds) ++ right.getPersistExprs(ds) - } - - override def getGroupbyExprPairs(dsPair: (String, String)): Seq[(Expr, Expr)] = { - if (compare == "=" || compare == "==") { - (left.dataSourceOpt, right.dataSourceOpt) match { - case (Some(dsPair._1), Some(dsPair._2)) => (left, right) :: Nil - case (Some(dsPair._2), Some(dsPair._1)) => (right, left) :: Nil - case _ => Nil - } - } else Nil - } -} - -case class LogicalRangeExpr(left: MathExpr, rangeOpr: String, range: RangeDesc) extends LogicalExpr { - private val (inOpr, ninOpr, btwnOpr, nbtwnOpr) = ("""(?i)in""".r, """(?i)not\s+in""".r, """(?i)between""".r, """(?i)not\s+between""".r) - def calculateOnly(values: Map[String, Any]): Option[Any] = { - val (lv, rvs) = (left.calculate(values), range.elements.map(_.calculate(values))) - rangeOpr match { - case this.inOpr() => lv in rvs - case this.ninOpr() => lv not_in rvs - case this.btwnOpr() => lv between rvs - case this.nbtwnOpr() => lv not_between rvs - case _ => None - } - } - val desc: String = s"${left.desc} ${rangeOpr} ${range.desc}" - val dataSources: Set[String] = left.dataSources ++ range.elements.flatMap(_.dataSources).toSet - override def getSubCacheExprs(ds: String): Iterable[Expr] = { - left.getCacheExprs(ds) ++ range.elements.flatMap(_.getCacheExprs(ds)) - } - override def getSubFinalCacheExprs(ds: String): Iterable[Expr] = { - left.getFinalCacheExprs(ds) ++ range.elements.flatMap(_.getFinalCacheExprs(ds)) - } - override def getSubPersistExprs(ds: String): Iterable[Expr] = { - left.getPersistExprs(ds) ++ range.elements.flatMap(_.getPersistExprs(ds)) - } -} - -// -- logical statement -- -//case class LogicalFactorExpr(self: LogicalExpr) extends LogicalExpr { -// def calculate(values: Map[String, Any]): Option[Any] = self.calculate(values) -// val desc: String = self.desc -//} - -case class UnaryLogicalExpr(oprList: Iterable[String], factor: LogicalExpr) extends LogicalExpr { - private val notOpr = """(?i)not|!""".r - def calculateOnly(values: Map[String, Any]): Option[Any] = { - val fv = factor.calculate(values) - oprList.foldRight(fv) { (opr, v) => - opr match { - case this.notOpr() => !v - case _ => None - } - } - } - val desc: String = oprList.foldRight(factor.desc) { (prev, ex) => s"${prev} ${ex}" } - val dataSources: Set[String] = factor.dataSources - override def getSubCacheExprs(ds: String): Iterable[Expr] = { - factor.getCacheExprs(ds) - } - override def getSubFinalCacheExprs(ds: String): Iterable[Expr] = { - factor.getFinalCacheExprs(ds) - } - override def getSubPersistExprs(ds: String): Iterable[Expr] = { - factor.getPersistExprs(ds) - } - - override def getGroupbyExprPairs(dsPair: (String, String)): Seq[(Expr, Expr)] = { - val notOprList = oprList.filter { opr => - opr match { - case this.notOpr() => true - case _ => false - } - } - if (notOprList.size % 2 == 0) factor.getGroupbyExprPairs(dsPair) else Nil - } -} - -case class BinaryLogicalExpr(first: LogicalExpr, others: Iterable[(String, LogicalExpr)]) extends LogicalExpr { - private val (andOpr, orOpr) = ("""(?i)and|&&""".r, """(?i)or|\|\|""".r) - def calculateOnly(values: Map[String, Any]): Option[Any] = { - val fv = first.calculate(values) - others.foldLeft(fv) { (v, pair) => - val (opr, next) = pair - val nv = next.calculate(values) - opr match { - case this.andOpr() => v && nv - case this.orOpr() => v || nv - case _ => None - } - } - } - val desc: String = others.foldLeft(first.desc) { (ex, next) => s"${ex} ${next._1} ${next._2.desc}" } - val dataSources: Set[String] = first.dataSources ++ others.flatMap(_._2.dataSources).toSet - override def getSubCacheExprs(ds: String): Iterable[Expr] = { - first.getCacheExprs(ds) ++ others.flatMap(_._2.getCacheExprs(ds)) - } - override def getSubFinalCacheExprs(ds: String): Iterable[Expr] = { - first.getFinalCacheExprs(ds) ++ others.flatMap(_._2.getFinalCacheExprs(ds)) - } - override def getSubPersistExprs(ds: String): Iterable[Expr] = { - first.getPersistExprs(ds) ++ others.flatMap(_._2.getPersistExprs(ds)) - } - - override def getGroupbyExprPairs(dsPair: (String, String)): Seq[(Expr, Expr)] = { - if (others.isEmpty) first.getGroupbyExprPairs(dsPair) - else { - val isAnd = others.exists(_._1 match { - case this.andOpr() => true - case _ => false - }) - if (isAnd) { - first.getGroupbyExprPairs(dsPair) ++ others.flatMap(_._2.getGroupbyExprPairs(dsPair)) - } else Nil - } - } -} \ No newline at end of file diff --git a/measure/src/main/scala/org/apache/griffin/measure/rule/expr/MathExpr.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/expr/MathExpr.scala deleted file mode 100644 index 661e8f45c..000000000 --- a/measure/src/main/scala/org/apache/griffin/measure/rule/expr/MathExpr.scala +++ /dev/null @@ -1,99 +0,0 @@ -/* -Licensed to the Apache Software Foundation (ASF) under one -or more contributor license agreements. See the NOTICE file -distributed with this work for additional information -regarding copyright ownership. The ASF licenses this file -to you under the Apache License, Version 2.0 (the -"License"); you may not use this file except in compliance -with the License. You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, -software distributed under the License is distributed on an -"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -KIND, either express or implied. See the License for the -specific language governing permissions and limitations -under the License. -*/ -package org.apache.griffin.measure.rule.expr - -import org.apache.griffin.measure.rule.CalculationUtil._ -import org.apache.griffin.measure.rule.DataTypeCalculationUtil._ -import org.apache.spark.sql.types.DataType - -trait MathExpr extends Expr { - -} - -case class MathFactorExpr(self: Expr) extends MathExpr { - def calculateOnly(values: Map[String, Any]): Option[Any] = self.calculate(values) - val desc: String = self.desc - val dataSources: Set[String] = self.dataSources - override def getSubCacheExprs(ds: String): Iterable[Expr] = { - self.getCacheExprs(ds) - } - override def getSubFinalCacheExprs(ds: String): Iterable[Expr] = { - self.getFinalCacheExprs(ds) - } - override def getSubPersistExprs(ds: String): Iterable[Expr] = { - self.getPersistExprs(ds) - } -} - -case class UnaryMathExpr(oprList: Iterable[String], factor: Expr) extends MathExpr { - private val (posOpr, negOpr) = ("+", "-") - def calculateOnly(values: Map[String, Any]): Option[Any] = { - val fv = factor.calculate(values) - oprList.foldRight(fv) { (opr, v) => - opr match { - case this.posOpr => v - case this.negOpr => -v - case _ => None - } - } - } - val desc: String = oprList.foldRight(factor.desc) { (prev, ex) => s"${prev}${ex}" } - val dataSources: Set[String] = factor.dataSources - override def cacheUnit: Boolean = true - override def getSubCacheExprs(ds: String): Iterable[Expr] = { - factor.getCacheExprs(ds) - } - override def getSubFinalCacheExprs(ds: String): Iterable[Expr] = { - factor.getFinalCacheExprs(ds) - } - override def getSubPersistExprs(ds: String): Iterable[Expr] = { - factor.getPersistExprs(ds) - } -} - -case class BinaryMathExpr(first: MathExpr, others: Iterable[(String, MathExpr)]) extends MathExpr { - private val (addOpr, subOpr, mulOpr, divOpr, modOpr) = ("+", "-", "*", "/", "%") - def calculateOnly(values: Map[String, Any]): Option[Any] = { - val fv = first.calculate(values) - others.foldLeft(fv) { (v, pair) => - val (opr, next) = pair - val nv = next.calculate(values) - opr match { - case this.addOpr => v + nv - case this.subOpr => v - nv - case this.mulOpr => v * nv - case this.divOpr => v / nv - case this.modOpr => v % nv - case _ => None - } - } - } - val desc: String = others.foldLeft(first.desc) { (ex, next) => s"${ex} ${next._1} ${next._2.desc}" } - val dataSources: Set[String] = first.dataSources ++ others.flatMap(_._2.dataSources).toSet - override def cacheUnit: Boolean = true - override def getSubCacheExprs(ds: String): Iterable[Expr] = { - first.getCacheExprs(ds) ++ others.flatMap(_._2.getCacheExprs(ds)) - } - override def getSubFinalCacheExprs(ds: String): Iterable[Expr] = { - first.getFinalCacheExprs(ds) ++ others.flatMap(_._2.getFinalCacheExprs(ds)) - } - override def getSubPersistExprs(ds: String): Iterable[Expr] = { - first.getPersistExprs(ds) ++ others.flatMap(_._2.getPersistExprs(ds)) - } -} \ No newline at end of file diff --git a/measure/src/main/scala/org/apache/griffin/measure/rule/expr/SelectExpr.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/expr/SelectExpr.scala deleted file mode 100644 index 5b7f1b00b..000000000 --- a/measure/src/main/scala/org/apache/griffin/measure/rule/expr/SelectExpr.scala +++ /dev/null @@ -1,88 +0,0 @@ -/* -Licensed to the Apache Software Foundation (ASF) under one -or more contributor license agreements. See the NOTICE file -distributed with this work for additional information -regarding copyright ownership. The ASF licenses this file -to you under the Apache License, Version 2.0 (the -"License"); you may not use this file except in compliance -with the License. You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, -software distributed under the License is distributed on an -"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -KIND, either express or implied. See the License for the -specific language governing permissions and limitations -under the License. -*/ -package org.apache.griffin.measure.rule.expr - -import org.apache.spark.sql.types.DataType -import org.apache.griffin.measure.rule.CalculationUtil._ - -trait SelectExpr extends Expr { - def calculateOnly(values: Map[String, Any]): Option[Any] = None -} - -case class IndexFieldRangeSelectExpr(fields: Iterable[FieldDescOnly]) extends SelectExpr { - val desc: String = s"[${fields.map(_.desc).mkString(", ")}]" - val dataSources: Set[String] = Set.empty[String] -} - -case class FunctionOperationExpr(func: String, args: Iterable[MathExpr]) extends SelectExpr { - val desc: String = s".${func}(${args.map(_.desc).mkString(", ")})" - val dataSources: Set[String] = args.flatMap(_.dataSources).toSet - override def getSubCacheExprs(ds: String): Iterable[Expr] = args.flatMap(_.getCacheExprs(ds)) - override def getSubFinalCacheExprs(ds: String): Iterable[Expr] = args.flatMap(_.getFinalCacheExprs(ds)) - override def getSubPersistExprs(ds: String): Iterable[Expr] = args.flatMap(_.getPersistExprs(ds)) -} - -case class FilterSelectExpr(field: FieldDesc, compare: String, value: MathExpr) extends SelectExpr { - val desc: String = s"[${field.desc} ${compare} ${value.desc}]" - val dataSources: Set[String] = value.dataSources - override def getSubCacheExprs(ds: String): Iterable[Expr] = value.getCacheExprs(ds) - override def getSubFinalCacheExprs(ds: String): Iterable[Expr] = value.getFinalCacheExprs(ds) - override def getSubPersistExprs(ds: String): Iterable[Expr] = value.getPersistExprs(ds) - private val (eqOpr, neqOpr, btOpr, bteOpr, ltOpr, lteOpr) = ("""==?""".r, """!==?""".r, ">", ">=", "<", "<=") - override def calculateOnly(values: Map[String, Any]): Option[Any] = { - val (lv, rv) = (values.get(fieldKey), value.calculate(values)) - compare match { - case this.eqOpr() => lv === rv - case this.neqOpr() => lv =!= rv - case this.btOpr => lv > rv - case this.bteOpr => lv >= rv - case this.ltOpr => lv < rv - case this.lteOpr => lv <= rv - case _ => None - } - } - def fieldKey: String = s"__${field.field}" -} - -// -- selection -- -case class SelectionExpr(head: SelectionHead, selectors: Iterable[SelectExpr]) extends Expr { - def calculateOnly(values: Map[String, Any]): Option[Any] = values.get(_id) - - val desc: String = { - val argsString = selectors.map(_.desc).mkString("") - s"${head.desc}${argsString}" - } - val dataSources: Set[String] = { - val selectorDataSources = selectors.flatMap(_.dataSources).toSet - selectorDataSources + head.head - } - - override def cacheUnit: Boolean = true - override def getSubCacheExprs(ds: String): Iterable[Expr] = { - selectors.flatMap(_.getCacheExprs(ds)) - } - override def getSubFinalCacheExprs(ds: String): Iterable[Expr] = { - selectors.flatMap(_.getFinalCacheExprs(ds)) - } - - override def persistUnit: Boolean = true - override def getSubPersistExprs(ds: String): Iterable[Expr] = { - selectors.flatMap(_.getPersistExprs(ds)) - } -} \ No newline at end of file diff --git a/measure/src/main/scala/org/apache/griffin/measure/rule/func/DefaultFunctionDefine.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/func/DefaultFunctionDefine.scala deleted file mode 100644 index 15161c36c..000000000 --- a/measure/src/main/scala/org/apache/griffin/measure/rule/func/DefaultFunctionDefine.scala +++ /dev/null @@ -1,36 +0,0 @@ -/* -Licensed to the Apache Software Foundation (ASF) under one -or more contributor license agreements. See the NOTICE file -distributed with this work for additional information -regarding copyright ownership. The ASF licenses this file -to you under the Apache License, Version 2.0 (the -"License"); you may not use this file except in compliance -with the License. You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, -software distributed under the License is distributed on an -"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -KIND, either express or implied. See the License for the -specific language governing permissions and limitations -under the License. -*/ -package org.apache.griffin.measure.rule.func - -import org.apache.griffin.measure.utils.JsonUtil - -class DefaultFunctionDefine extends FunctionDefine { - - def json(strOpt: Option[_]): Map[String, Any] = { - try { - strOpt match { - case Some(str: String) => JsonUtil.toAnyMap(str) - case _ => throw new Exception("json function param should be string") - } - } catch { - case e: Throwable => throw e - } - } - -} diff --git a/measure/src/main/scala/org/apache/griffin/measure/rule/func/FunctionDefine.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/func/FunctionDefine.scala deleted file mode 100644 index d23fc7af3..000000000 --- a/measure/src/main/scala/org/apache/griffin/measure/rule/func/FunctionDefine.scala +++ /dev/null @@ -1,25 +0,0 @@ -/* -Licensed to the Apache Software Foundation (ASF) under one -or more contributor license agreements. See the NOTICE file -distributed with this work for additional information -regarding copyright ownership. The ASF licenses this file -to you under the Apache License, Version 2.0 (the -"License"); you may not use this file except in compliance -with the License. You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, -software distributed under the License is distributed on an -"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -KIND, either express or implied. See the License for the -specific language governing permissions and limitations -under the License. -*/ -package org.apache.griffin.measure.rule.func - -trait FunctionDefine extends Serializable { - -} - -class UnKnown {} \ No newline at end of file diff --git a/measure/src/main/scala/org/apache/griffin/measure/rule/func/FunctionUtil.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/func/FunctionUtil.scala deleted file mode 100644 index 57e934d6a..000000000 --- a/measure/src/main/scala/org/apache/griffin/measure/rule/func/FunctionUtil.scala +++ /dev/null @@ -1,75 +0,0 @@ -/* -Licensed to the Apache Software Foundation (ASF) under one -or more contributor license agreements. See the NOTICE file -distributed with this work for additional information -regarding copyright ownership. The ASF licenses this file -to you under the Apache License, Version 2.0 (the -"License"); you may not use this file except in compliance -with the License. You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, -software distributed under the License is distributed on an -"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -KIND, either express or implied. See the License for the -specific language governing permissions and limitations -under the License. -*/ -package org.apache.griffin.measure.rule.func - -import java.lang.reflect.Method - -import org.apache.griffin.measure.log.Loggable - -import scala.collection.mutable.{Map => MutableMap} - -object FunctionUtil extends Loggable { - - val functionDefines: MutableMap[String, FunctionDefine] = MutableMap[String, FunctionDefine]() - - registerFunctionDefine(Array(classOf[DefaultFunctionDefine].getCanonicalName)) - - def registerFunctionDefine(classes: Iterable[String]): Unit = { - for (cls <- classes) { - try { - val clz: Class[_] = Class.forName(cls) - if (classOf[FunctionDefine].isAssignableFrom(clz)) { - functionDefines += (cls -> clz.newInstance.asInstanceOf[FunctionDefine]) - } else { - warn(s"${cls} register fails: ${cls} is not sub class of ${classOf[FunctionDefine].getCanonicalName}") - } - } catch { - case e: Throwable => warn(s"${cls} register fails: ${e.getMessage}") - } - } - } - - def invoke(methodName: String, params: Array[Option[Any]]): Seq[Option[Any]] = { -// val paramTypes = params.map { param => -// try { -// param match { -// case Some(v) => v.getClass -// case _ => classOf[UnKnown] -// } -// } catch { -// case e: Throwable => classOf[UnKnown] -// } -// } - val paramTypes = params.map(a => classOf[Option[_]]) - - functionDefines.values.foldLeft(Nil: Seq[Option[Any]]) { (res, funcDef) => - if (res.isEmpty) { - val clz = funcDef.getClass - try { - val method = clz.getMethod(methodName, paramTypes: _*) - Seq(Some(method.invoke(funcDef, params: _*))) - } catch { - case e: Throwable => res - } - } else res - } - } - -} - diff --git a/measure/src/main/scala/org/apache/griffin/measure/rules/preproc/PreProcRuleGenerator.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/preproc/PreProcRuleGenerator.scala similarity index 97% rename from measure/src/main/scala/org/apache/griffin/measure/rules/preproc/PreProcRuleGenerator.scala rename to measure/src/main/scala/org/apache/griffin/measure/rule/preproc/PreProcRuleGenerator.scala index 571841c47..22d64d8af 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rules/preproc/PreProcRuleGenerator.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rule/preproc/PreProcRuleGenerator.scala @@ -16,7 +16,7 @@ KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License. */ -package org.apache.griffin.measure.rules.preproc +package org.apache.griffin.measure.rule.preproc object PreProcRuleGenerator { diff --git a/measure/src/main/scala/org/apache/griffin/measure/rules/step/ConcreteRuleStep.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/step/ConcreteRuleStep.scala similarity index 92% rename from measure/src/main/scala/org/apache/griffin/measure/rules/step/ConcreteRuleStep.scala rename to measure/src/main/scala/org/apache/griffin/measure/rule/step/ConcreteRuleStep.scala index f9b6b9098..4b3a4d4c2 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rules/step/ConcreteRuleStep.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rule/step/ConcreteRuleStep.scala @@ -16,9 +16,9 @@ KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License. */ -package org.apache.griffin.measure.rules.step +package org.apache.griffin.measure.rule.step -import org.apache.griffin.measure.rules.dsl._ +import org.apache.griffin.measure.rule.dsl._ trait ConcreteRuleStep extends RuleStep { diff --git a/measure/src/main/scala/org/apache/griffin/measure/rules/step/DfOprStep.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/step/DfOprStep.scala similarity index 91% rename from measure/src/main/scala/org/apache/griffin/measure/rules/step/DfOprStep.scala rename to measure/src/main/scala/org/apache/griffin/measure/rule/step/DfOprStep.scala index 5a85ea91c..86f0bf396 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rules/step/DfOprStep.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rule/step/DfOprStep.scala @@ -16,9 +16,9 @@ KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License. */ -package org.apache.griffin.measure.rules.step +package org.apache.griffin.measure.rule.step -import org.apache.griffin.measure.rules.dsl._ +import org.apache.griffin.measure.rule.dsl._ case class DfOprStep(name: String, rule: String, details: Map[String, Any], persistType: PersistType, updateDataSource: Option[String] diff --git a/measure/src/main/scala/org/apache/griffin/measure/rules/step/GriffinDslStep.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/step/GriffinDslStep.scala similarity index 91% rename from measure/src/main/scala/org/apache/griffin/measure/rules/step/GriffinDslStep.scala rename to measure/src/main/scala/org/apache/griffin/measure/rule/step/GriffinDslStep.scala index 68c2c9a1c..21db8cff9 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rules/step/GriffinDslStep.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rule/step/GriffinDslStep.scala @@ -16,9 +16,9 @@ KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License. */ -package org.apache.griffin.measure.rules.step +package org.apache.griffin.measure.rule.step -import org.apache.griffin.measure.rules.dsl._ +import org.apache.griffin.measure.rule.dsl._ case class GriffinDslStep(name: String, rule: String, dqType: DqType, details: Map[String, Any] ) extends RuleStep { diff --git a/measure/src/main/scala/org/apache/griffin/measure/rules/step/RuleStep.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/step/RuleStep.scala similarity index 88% rename from measure/src/main/scala/org/apache/griffin/measure/rules/step/RuleStep.scala rename to measure/src/main/scala/org/apache/griffin/measure/rule/step/RuleStep.scala index 9258be067..4675ffe7d 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rules/step/RuleStep.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rule/step/RuleStep.scala @@ -16,9 +16,9 @@ KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License. */ -package org.apache.griffin.measure.rules.step +package org.apache.griffin.measure.rule.step -import org.apache.griffin.measure.rules.dsl.{DslType, PersistType} +import org.apache.griffin.measure.rule.dsl.{DslType, PersistType} trait RuleStep extends Serializable { diff --git a/measure/src/main/scala/org/apache/griffin/measure/rules/step/SparkSqlStep.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/step/SparkSqlStep.scala similarity index 92% rename from measure/src/main/scala/org/apache/griffin/measure/rules/step/SparkSqlStep.scala rename to measure/src/main/scala/org/apache/griffin/measure/rule/step/SparkSqlStep.scala index 943cce55e..62c3c350f 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rules/step/SparkSqlStep.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rule/step/SparkSqlStep.scala @@ -16,10 +16,10 @@ KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License. */ -package org.apache.griffin.measure.rules.step +package org.apache.griffin.measure.rule.step import org.apache.griffin.measure.persist._ -import org.apache.griffin.measure.rules.dsl._ +import org.apache.griffin.measure.rule.dsl._ case class SparkSqlStep(name: String, rule: String, details: Map[String, Any], persistType: PersistType, updateDataSource: Option[String] diff --git a/measure/src/main/scala/org/apache/griffin/measure/rules/udf/GriffinUdfs.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/udf/GriffinUdfs.scala similarity index 95% rename from measure/src/main/scala/org/apache/griffin/measure/rules/udf/GriffinUdfs.scala rename to measure/src/main/scala/org/apache/griffin/measure/rule/udf/GriffinUdfs.scala index a4bded366..11e8c8fe4 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rules/udf/GriffinUdfs.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rule/udf/GriffinUdfs.scala @@ -16,7 +16,7 @@ KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License. */ -package org.apache.griffin.measure.rules.udf +package org.apache.griffin.measure.rule.udf import org.apache.spark.sql.SQLContext diff --git a/measure/src/test/resources/config-test-profiling-streaming.json b/measure/src/test/resources/config-test-profiling-streaming.json index 8a5e31f26..5a35230a0 100644 --- a/measure/src/test/resources/config-test-profiling-streaming.json +++ b/measure/src/test/resources/config-test-profiling-streaming.json @@ -54,7 +54,7 @@ { "dsl.type": "griffin-dsl", "dq.type": "profiling", - "rule": "source.name.count() where source.age > 5", + "rule": "source.name.count(), source.age.avg(), source.age.max(), source.age.min()", "details": { "source": "source", "profiling": { diff --git a/measure/src/test/scala/org/apache/griffin/measure/algo/batch/BatchAccuracyAlgoTest.scala b/measure/src/test/scala/org/apache/griffin/measure/algo/batch/BatchAccuracyAlgoTest.scala deleted file mode 100644 index 3e1551195..000000000 --- a/measure/src/test/scala/org/apache/griffin/measure/algo/batch/BatchAccuracyAlgoTest.scala +++ /dev/null @@ -1,294 +0,0 @@ -///* -//Licensed to the Apache Software Foundation (ASF) under one -//or more contributor license agreements. See the NOTICE file -//distributed with this work for additional information -//regarding copyright ownership. The ASF licenses this file -//to you under the Apache License, Version 2.0 (the -//"License"); you may not use this file except in compliance -//with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -//Unless required by applicable law or agreed to in writing, -//software distributed under the License is distributed on an -//"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -//KIND, either express or implied. See the License for the -//specific language governing permissions and limitations -//under the License. -//*/ -//package org.apache.griffin.measure.algo -// -//import java.util.Date -// -//import org.apache.griffin.measure.algo.batch.BatchAccuracyAlgo -//import org.apache.griffin.measure.config.params._ -//import org.apache.griffin.measure.config.params.env._ -//import org.apache.griffin.measure.config.params.user._ -//import org.apache.griffin.measure.config.reader._ -//import org.apache.griffin.measure.config.validator._ -//import org.apache.griffin.measure.data.connector.direct.DirectDataConnector -//import org.apache.griffin.measure.data.connector.{DataConnector, DataConnectorFactory} -//import org.apache.griffin.measure.log.Loggable -//import org.apache.griffin.measure.rule.expr._ -//import org.apache.griffin.measure.rule.{ExprValueUtil, RuleAnalyzer, RuleFactory} -//import org.apache.spark.rdd.RDD -//import org.apache.spark.sql.SQLContext -//import org.apache.spark.sql.hive.HiveContext -//import org.apache.spark.{SparkConf, SparkContext} -//import org.junit.runner.RunWith -//import org.scalatest.junit.JUnitRunner -//import org.scalatest.{BeforeAndAfter, FunSuite, Matchers} -// -//import scala.util.{Failure, Success, Try} -// -// -//@RunWith(classOf[JUnitRunner]) -//class BatchAccuracyAlgoTest extends FunSuite with Matchers with BeforeAndAfter with Loggable { -// -// val envFile = "src/test/resources/env.json" -// val confFile = "src/test/resources/config.json" -//// val confFile = "{\"name\":\"accu1\",\"type\":\"accuracy\",\"source\":{\"type\":\"avro\",\"version\":\"1.7\",\"config\":{\"file.name\":\"src/test/resources/users_info_src.avro\"}},\"target\":{\"type\":\"avro\",\"version\":\"1.7\",\"config\":{\"file.name\":\"src/test/resources/users_info_target.avro\"}},\"evaluateRule\":{\"sampleRatio\":1,\"rules\":\"$source.user_id + 5 = $target.user_id + (2 + 3) AND $source.first_name + 12 = $target.first_name + (10 + 2) 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 AND (15 OR true) WHEN true AND $source.user_id > 10020\"}}" -// val envFsType = "local" -// val userFsType = "local" -// -// val args = Array(envFile, confFile) -// -// var sc: SparkContext = _ -// var sqlContext: SQLContext = _ -// -// var allParam: AllParam = _ -// -// before { -// // read param files -// val envParam = readParamFile[EnvParam](envFile, envFsType) match { -// case Success(p) => p -// case Failure(ex) => { -// error(ex.getMessage) -// sys.exit(-2) -// } -// } -// val userParam = readParamFile[UserParam](confFile, userFsType) match { -// case Success(p) => p -// case Failure(ex) => { -// error(ex.getMessage) -// sys.exit(-2) -// } -// } -// allParam = AllParam(envParam, userParam) -// -// // validate param files -// validateParams(allParam) match { -// case Failure(ex) => { -// error(ex.getMessage) -// sys.exit(-3) -// } -// case _ => { -// info("params validation pass") -// } -// } -// -// val metricName = userParam.name -// val conf = new SparkConf().setMaster("local[*]").setAppName(metricName) -// sc = new SparkContext(conf) -//// sqlContext = new SQLContext(sc) -// sqlContext = new HiveContext(sc) -// } -// -// test("algorithm") { -// Try { -// val envParam = allParam.envParam -// val userParam = allParam.userParam -// -// // start time -// val startTime = new Date().getTime() -// -// // get spark application id -// val applicationId = sc.applicationId -// -// // rules -// val ruleFactory = RuleFactory(userParam.evaluateRuleParam) -// val rule: StatementExpr = ruleFactory.generateRule() -// val ruleAnalyzer: RuleAnalyzer = RuleAnalyzer(rule) -// -// ruleAnalyzer.constCacheExprs.foreach(println) -// ruleAnalyzer.constFinalCacheExprs.foreach(println) -// -// // global cache data -// val constExprValueMap = ExprValueUtil.genExprValueMaps(None, ruleAnalyzer.constCacheExprs, Map[String, Any]()) -// val finalConstExprValueMap = ExprValueUtil.updateExprValueMaps(ruleAnalyzer.constFinalCacheExprs, constExprValueMap) -// val finalConstMap = finalConstExprValueMap.headOption match { -// case Some(m) => m -// case _ => Map[String, Any]() -// } -// -// // data connector -// val sourceDataConnector: DirectDataConnector = -// DataConnectorFactory.getDirectDataConnector(sqlContext, null, userParam.sourceParam, -// ruleAnalyzer.sourceRuleExprs, finalConstMap -// ) match { -// case Success(cntr) => { -// if (cntr.available) cntr -// else throw new Exception("source data not available!") -// } -// case Failure(ex) => throw ex -// } -// val targetDataConnector: DirectDataConnector = -// DataConnectorFactory.getDirectDataConnector(sqlContext, null, userParam.targetParam, -// ruleAnalyzer.targetRuleExprs, finalConstMap -// ) match { -// case Success(cntr) => { -// if (cntr.available) cntr -// else throw new Exception("target data not available!") -// } -// case Failure(ex) => throw ex -// } -// -// // get metadata -//// val sourceMetaData: Iterable[(String, String)] = sourceDataConnector.metaData() match { -//// case Success(md) => md -//// case Failure(ex) => throw ex -//// } -//// val targetMetaData: Iterable[(String, String)] = targetDataConnector.metaData() match { -//// case Success(md) => md -//// case Failure(ex) => throw ex -//// } -// -// // get data -// val sourceData: RDD[(Product, (Map[String, Any], Map[String, Any]))] = sourceDataConnector.data() match { -// case Success(dt) => dt -// case Failure(ex) => throw ex -// } -// val targetData: RDD[(Product, (Map[String, Any], Map[String, Any]))] = targetDataConnector.data() match { -// case Success(dt) => dt -// case Failure(ex) => throw ex -// } -// -// // my algo -// val algo = BatchAccuracyAlgo(allParam) -// -// // accuracy algorithm -// val (accuResult, missingRdd, matchedRdd) = algo.accuracy(sourceData, targetData, ruleAnalyzer) -// -// println(s"match percentage: ${accuResult.matchPercentage}, total count: ${accuResult.total}") -// -// missingRdd.map(rec => algo.record2String(rec, ruleAnalyzer.sourceRuleExprs.persistExprs, ruleAnalyzer.targetRuleExprs.persistExprs)).foreach(println) -// -// // end time -// val endTime = new Date().getTime -// println(s"using time: ${endTime - startTime} ms") -// } match { -// case Failure(ex) => { -// error(ex.getMessage) -// sys.exit(-4) -// } -// case _ => { -// info("calculation finished") -// } -// } -// } -// -// private def readParamFile[T <: Param](file: String, fsType: String)(implicit m : Manifest[T]): Try[T] = { -// val paramReader = ParamReaderFactory.getParamReader(file, fsType) -// paramReader.readConfig[T] -// } -// -// private def validateParams(allParam: AllParam): Try[Boolean] = { -// val allParamValidator = AllParamValidator() -// allParamValidator.validate(allParam) -// } -// -// test ("spark sql") { -// Try { -// val envParam = allParam.envParam -// val userParam = allParam.userParam -// -// // start time -// val startTime = new Date().getTime() -// -// // get spark application id -// val applicationId = sc.applicationId -// -//// val sourceFilePath = "src/test/resources/users_info_src.avro" -//// val targetFilePath = "src/test/resources/users_info_target.avro" -//// -//// val sourceDF = sqlContext.read.format("com.databricks.spark.avro").load(sourceFilePath) -//// val targetDF = sqlContext.read.format("com.databricks.spark.avro").load(targetFilePath) -// -// val sourceTableName = "source.table" -// val targetTableName = "target.table" -// -// val sourceDF = sqlContext.sql("SELECT * FROM default.data_avr") -// val targetDF = sqlContext.sql("SELECT * FROM default.data_only") -// -//// sourceDF.show(100) -//// targetDF.show(100) -// -// sourceDF.registerTempTable(sourceTableName) -// targetDF.registerTempTable(targetTableName) -// -//// val sourceTableName = "data_avr" -//// val targetTableName = "data_avr" -// -//// val sql = -//// s""" -//// |SELECT COUNT(*) FROM `${sourceTableName}` LEFT JOIN `${targetTableName}` -//// |ON `${sourceTableName}`.uid = `${targetTableName}`.uid -//// """.stripMargin -// -// val sql = -// s""" -// |SELECT `${sourceTableName}`.uid, `${sourceTableName}`.uage, `${sourceTableName}`.udes, -// |`${targetTableName}`.uid, `${targetTableName}`.uage, `${targetTableName}`.udes -// |FROM `${sourceTableName}` LEFT JOIN `${targetTableName}` -// |ON coalesce(`${sourceTableName}`.uid, 'null') = coalesce(`${targetTableName}`.uid, 'null') -// |AND coalesce(`${sourceTableName}`.uage, 'null') = coalesce(`${targetTableName}`.uage, 'null') -// |AND coalesce(`${sourceTableName}`.udes, 'null') = coalesce(`${targetTableName}`.udes, 'null') -// |WHERE (NOT (`${sourceTableName}`.uid IS NULL -// |AND `${sourceTableName}`.uage IS NULL -// |AND `${sourceTableName}`.udes IS NULL)) -// |AND ((`${targetTableName}`.uid IS NULL -// |AND `${targetTableName}`.uage IS NULL -// |AND `${targetTableName}`.udes IS NULL)) -// """.stripMargin -// -//// val sql = -//// """ -//// |SELECT * FROM source LEFT JOIN target -//// |ON source.user_id = target.user_id -//// |AND source.first_name = 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 coalesce(source.post_code, 'null') = coalesce(target.post_code, 'null') -//// """.stripMargin -// -//// val sql = -//// """ -//// |SELECT * FROM source WHERE source.post_code IS NULL -//// """.stripMargin -// -// val result = sqlContext.sql(sql) -// -// result.show(100) -// -//// result.registerTempTable("result") -//// val rsql = "SELECT COUNT(*) FROM result" -//// val rr = sqlContext.sql(rsql) -//// rr.show(100) -// -// // end time -// val endTime = new Date().getTime -// println(s"using time: ${endTime - startTime} ms") -// } match { -// case Failure(ex) => { -// error(ex.getMessage) -// sys.exit(-4) -// } -// case _ => { -// info("calculation finished") -// } -// } -// } -// -//} diff --git a/measure/src/test/scala/org/apache/griffin/measure/algo/batch/BatchProfileAlgoTest.scala b/measure/src/test/scala/org/apache/griffin/measure/algo/batch/BatchProfileAlgoTest.scala deleted file mode 100644 index 8636d0262..000000000 --- a/measure/src/test/scala/org/apache/griffin/measure/algo/batch/BatchProfileAlgoTest.scala +++ /dev/null @@ -1,173 +0,0 @@ -///* -//Licensed to the Apache Software Foundation (ASF) under one -//or more contributor license agreements. See the NOTICE file -//distributed with this work for additional information -//regarding copyright ownership. The ASF licenses this file -//to you under the Apache License, Version 2.0 (the -//"License"); you may not use this file except in compliance -//with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -//Unless required by applicable law or agreed to in writing, -//software distributed under the License is distributed on an -//"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -//KIND, either express or implied. See the License for the -//specific language governing permissions and limitations -//under the License. -//*/ -//package org.apache.griffin.measure.algo -// -//import java.util.Date -// -//import org.apache.griffin.measure.algo.batch.BatchProfileAlgo -//import org.apache.griffin.measure.config.params._ -//import org.apache.griffin.measure.config.params.env._ -//import org.apache.griffin.measure.config.params.user._ -//import org.apache.griffin.measure.config.reader._ -//import org.apache.griffin.measure.config.validator._ -//import org.apache.griffin.measure.data.connector.direct.DirectDataConnector -//import org.apache.griffin.measure.data.connector.{DataConnector, DataConnectorFactory} -//import org.apache.griffin.measure.log.Loggable -//import org.apache.griffin.measure.rule.expr._ -//import org.apache.griffin.measure.rule.{ExprValueUtil, RuleAnalyzer, RuleFactory} -//import org.apache.spark.rdd.RDD -//import org.apache.spark.sql.SQLContext -//import org.apache.spark.{SparkConf, SparkContext} -//import org.junit.runner.RunWith -//import org.scalatest.junit.JUnitRunner -//import org.scalatest.{BeforeAndAfter, FunSuite, Matchers} -// -//import scala.util.{Failure, Success, Try} -// -// -//@RunWith(classOf[JUnitRunner]) -//class BatchProfileAlgoTest extends FunSuite with Matchers with BeforeAndAfter with Loggable { -// -// val envFile = "src/test/resources/env.json" -// val confFile = "src/test/resources/config-profile.json" -// val envFsType = "local" -// val userFsType = "local" -// -// val args = Array(envFile, confFile) -// -// var sc: SparkContext = _ -// var sqlContext: SQLContext = _ -// -// var allParam: AllParam = _ -// -// before { -// // read param files -// val envParam = readParamFile[EnvParam](envFile, envFsType) match { -// case Success(p) => p -// case Failure(ex) => { -// error(ex.getMessage) -// sys.exit(-2) -// } -// } -// val userParam = readParamFile[UserParam](confFile, userFsType) match { -// case Success(p) => p -// case Failure(ex) => { -// error(ex.getMessage) -// sys.exit(-2) -// } -// } -// allParam = AllParam(envParam, userParam) -// -// // validate param files -// validateParams(allParam) match { -// case Failure(ex) => { -// error(ex.getMessage) -// sys.exit(-3) -// } -// case _ => { -// info("params validation pass") -// } -// } -// -// val metricName = userParam.name -// val conf = new SparkConf().setMaster("local[*]").setAppName(metricName) -// sc = new SparkContext(conf) -// sqlContext = new SQLContext(sc) -// } -// -// test("algorithm") { -// Try { -// val envParam = allParam.envParam -// val userParam = allParam.userParam -// -// // start time -// val startTime = new Date().getTime() -// -// // get spark application id -// val applicationId = sc.applicationId -// -// // rules -// val ruleFactory = RuleFactory(userParam.evaluateRuleParam) -// val rule: StatementExpr = ruleFactory.generateRule() -// val ruleAnalyzer: RuleAnalyzer = RuleAnalyzer(rule) -// -// ruleAnalyzer.constCacheExprs.foreach(println) -// ruleAnalyzer.constFinalCacheExprs.foreach(println) -// -// // global cache data -// val constExprValueMap = ExprValueUtil.genExprValueMaps(None, ruleAnalyzer.constCacheExprs, Map[String, Any]()) -// val finalConstExprValueMap = ExprValueUtil.updateExprValueMaps(ruleAnalyzer.constFinalCacheExprs, constExprValueMap) -// val finalConstMap = finalConstExprValueMap.headOption match { -// case Some(m) => m -// case _ => Map[String, Any]() -// } -// -// // data connector -// val sourceDataConnector: DirectDataConnector = -// DataConnectorFactory.getDirectDataConnector(sqlContext, null, userParam.sourceParam, -// ruleAnalyzer.sourceRuleExprs, finalConstMap -// ) match { -// case Success(cntr) => { -// if (cntr.available) cntr -// else throw new Exception("source data not available!") -// } -// case Failure(ex) => throw ex -// } -// -// // get data -// val sourceData: RDD[(Product, (Map[String, Any], Map[String, Any]))] = sourceDataConnector.data() match { -// case Success(dt) => dt -// case Failure(ex) => throw ex -// } -// -// // my algo -// val algo = BatchProfileAlgo(allParam) -// -// // profile algorithm -// val (profileResult, missingRdd, matchedRdd) = algo.profile(sourceData, ruleAnalyzer) -// -// println(s"match percentage: ${profileResult.matchPercentage}, match count: ${profileResult.matchCount}, total count: ${profileResult.totalCount}") -// -// matchedRdd.map(rec => algo.record2String(rec, ruleAnalyzer.sourceRuleExprs.persistExprs)).foreach(println) -// -// // end time -// val endTime = new Date().getTime -// println(s"using time: ${endTime - startTime} ms") -// } match { -// case Failure(ex) => { -// error(ex.getMessage) -// sys.exit(-4) -// } -// case _ => { -// info("calculation finished") -// } -// } -// } -// -// private def readParamFile[T <: Param](file: String, fsType: String)(implicit m : Manifest[T]): Try[T] = { -// val paramReader = ParamReaderFactory.getParamReader(file, fsType) -// paramReader.readConfig[T] -// } -// -// private def validateParams(allParam: AllParam): Try[Boolean] = { -// val allParamValidator = AllParamValidator() -// allParamValidator.validate(allParam) -// } -// -//} diff --git a/measure/src/test/scala/org/apache/griffin/measure/algo/batch/DataFrameSaveTest.scala b/measure/src/test/scala/org/apache/griffin/measure/algo/batch/DataFrameSaveTest.scala deleted file mode 100644 index d73e45808..000000000 --- a/measure/src/test/scala/org/apache/griffin/measure/algo/batch/DataFrameSaveTest.scala +++ /dev/null @@ -1,172 +0,0 @@ -///* -//Licensed to the Apache Software Foundation (ASF) under one -//or more contributor license agreements. See the NOTICE file -//distributed with this work for additional information -//regarding copyright ownership. The ASF licenses this file -//to you under the Apache License, Version 2.0 (the -//"License"); you may not use this file except in compliance -//with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -//Unless required by applicable law or agreed to in writing, -//software distributed under the License is distributed on an -//"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -//KIND, either express or implied. See the License for the -//specific language governing permissions and limitations -//under the License. -//*/ -//package org.apache.griffin.measure.algo.batch -// -//import java.util.Date -// -//import org.apache.griffin.measure.config.params._ -//import org.apache.griffin.measure.config.params.env._ -//import org.apache.griffin.measure.config.params.user._ -//import org.apache.griffin.measure.config.reader._ -//import org.apache.griffin.measure.config.validator._ -//import org.apache.griffin.measure.data.connector.DataConnectorFactory -//import org.apache.griffin.measure.data.connector.direct.DirectDataConnector -//import org.apache.griffin.measure.log.Loggable -//import org.apache.griffin.measure.rule.expr._ -//import org.apache.griffin.measure.rule.{ExprValueUtil, RuleAnalyzer, RuleFactory} -//import org.apache.spark.rdd.RDD -//import org.apache.spark.sql.SQLContext -//import org.apache.spark.{SparkConf, SparkContext} -//import org.junit.runner.RunWith -//import org.scalatest.junit.JUnitRunner -//import org.scalatest.{BeforeAndAfter, FunSuite, Matchers} -// -//import scala.util.{Failure, Success, Try} -// -// -//@RunWith(classOf[JUnitRunner]) -//class DataFrameSaveTest extends FunSuite with Matchers with BeforeAndAfter with Loggable { -// -// val envFile = "src/test/resources/env.json" -// val confFile = "src/test/resources/config-profile.json" -// val envFsType = "local" -// val userFsType = "local" -// -// val args = Array(envFile, confFile) -// -// var sc: SparkContext = _ -// var sqlContext: SQLContext = _ -// -// var allParam: AllParam = _ -// -// before { -// // read param files -// val envParam = readParamFile[EnvParam](envFile, envFsType) match { -// case Success(p) => p -// case Failure(ex) => { -// error(ex.getMessage) -// sys.exit(-2) -// } -// } -// val userParam = readParamFile[UserParam](confFile, userFsType) match { -// case Success(p) => p -// case Failure(ex) => { -// error(ex.getMessage) -// sys.exit(-2) -// } -// } -// allParam = AllParam(envParam, userParam) -// -// // validate param files -// validateParams(allParam) match { -// case Failure(ex) => { -// error(ex.getMessage) -// sys.exit(-3) -// } -// case _ => { -// info("params validation pass") -// } -// } -// -// val metricName = userParam.name -// val conf = new SparkConf().setMaster("local[*]").setAppName(metricName) -// sc = new SparkContext(conf) -// sqlContext = new SQLContext(sc) -// } -// -// test("algorithm") { -// Try { -// val envParam = allParam.envParam -// val userParam = allParam.userParam -// -// // start time -// val startTime = new Date().getTime() -// -// // get spark application id -// val applicationId = sc.applicationId -// -// // rules -// val ruleFactory = RuleFactory(userParam.evaluateRuleParam) -// val rule: StatementExpr = ruleFactory.generateRule() -// val ruleAnalyzer: RuleAnalyzer = RuleAnalyzer(rule) -// -// ruleAnalyzer.constCacheExprs.foreach(println) -// ruleAnalyzer.constFinalCacheExprs.foreach(println) -// -// // global cache data -// val constExprValueMap = ExprValueUtil.genExprValueMaps(None, ruleAnalyzer.constCacheExprs, Map[String, Any]()) -// val finalConstExprValueMap = ExprValueUtil.updateExprValueMaps(ruleAnalyzer.constFinalCacheExprs, constExprValueMap) -// val finalConstMap = finalConstExprValueMap.headOption match { -// case Some(m) => m -// case _ => Map[String, Any]() -// } -// -// // data connector -// val sourceDataConnector: DirectDataConnector = -// DataConnectorFactory.getDirectDataConnector(sqlContext, null, userParam.sourceParam, -// ruleAnalyzer.sourceRuleExprs, finalConstMap -// ) match { -// case Success(cntr) => { -// if (cntr.available) cntr -// else throw new Exception("source data not available!") -// } -// case Failure(ex) => throw ex -// } -// -// // get data -// val sourceData: RDD[(Product, (Map[String, Any], Map[String, Any]))] = sourceDataConnector.data() match { -// case Success(dt) => dt -// case Failure(ex) => throw ex -// } -// -// // my algo -// val algo = BatchProfileAlgo(allParam) -// -// // profile algorithm -// val (profileResult, missingRdd, matchedRdd) = algo.profile(sourceData, ruleAnalyzer) -// -// println(s"match percentage: ${profileResult.matchPercentage}, match count: ${profileResult.matchCount}, total count: ${profileResult.totalCount}") -// -// matchedRdd.map(rec => algo.record2String(rec, ruleAnalyzer.sourceRuleExprs.persistExprs)).foreach(println) -// -// // end time -// val endTime = new Date().getTime -// println(s"using time: ${endTime - startTime} ms") -// } match { -// case Failure(ex) => { -// error(ex.getMessage) -// sys.exit(-4) -// } -// case _ => { -// info("calculation finished") -// } -// } -// } -// -// private def readParamFile[T <: Param](file: String, fsType: String)(implicit m : Manifest[T]): Try[T] = { -// val paramReader = ParamReaderFactory.getParamReader(file, fsType) -// paramReader.readConfig[T] -// } -// -// private def validateParams(allParam: AllParam): Try[Boolean] = { -// val allParamValidator = AllParamValidator() -// allParamValidator.validate(allParam) -// } -// -//} diff --git a/measure/src/test/scala/org/apache/griffin/measure/algo/core/AccuracyCoreTest.scala b/measure/src/test/scala/org/apache/griffin/measure/algo/core/AccuracyCoreTest.scala deleted file mode 100644 index 034add69b..000000000 --- a/measure/src/test/scala/org/apache/griffin/measure/algo/core/AccuracyCoreTest.scala +++ /dev/null @@ -1,89 +0,0 @@ -///* -//Licensed to the Apache Software Foundation (ASF) under one -//or more contributor license agreements. See the NOTICE file -//distributed with this work for additional information -//regarding copyright ownership. The ASF licenses this file -//to you under the Apache License, Version 2.0 (the -//"License"); you may not use this file except in compliance -//with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -//Unless required by applicable law or agreed to in writing, -//software distributed under the License is distributed on an -//"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -//KIND, either express or implied. See the License for the -//specific language governing permissions and limitations -//under the License. -//*/ -//package org.apache.griffin.measure.algo.core -// -//import org.apache.griffin.measure.config.params.user.EvaluateRuleParam -//import org.apache.griffin.measure.rule.expr._ -//import org.apache.griffin.measure.rule.{RuleAnalyzer, RuleFactory} -//import org.junit.runner.RunWith -//import org.scalatest.junit.JUnitRunner -//import org.scalatest.{BeforeAndAfter, FunSuite, Matchers} -//import org.scalatest.PrivateMethodTester -// -//@RunWith(classOf[JUnitRunner]) -//class AccuracyCoreTest extends FunSuite with Matchers with BeforeAndAfter with PrivateMethodTester { -// -// def findExprId(exprs: Iterable[Expr], desc: String): String = { -// exprs.find(_.desc == desc) match { -// case Some(expr) => expr._id -// case _ => "" -// } -// } -// -// test ("match data success") { -// val rule = "$source.name = $target.name AND $source.age < $target.age" -// val evaluateRuleParam = EvaluateRuleParam(1.0, rule) -// val ruleFactory = RuleFactory(evaluateRuleParam) -// val statement = ruleFactory.generateRule -// val ruleAnalyzer = RuleAnalyzer(statement) -// -// val sourcePersistExprs = ruleAnalyzer.sourceRuleExprs.persistExprs -// val targetPersistExprs = ruleAnalyzer.targetRuleExprs.persistExprs -// -// val source = (Map[String, Any]( -// (findExprId(sourcePersistExprs, "$source['name']") -> "jack"), -// (findExprId(sourcePersistExprs, "$source['age']") -> 26) -// ), Map[String, Any]()) -// val target = (Map[String, Any]( -// (findExprId(targetPersistExprs, "$target['name']") -> "jack"), -// (findExprId(targetPersistExprs, "$target['age']") -> 27) -// ), Map[String, Any]()) -// -// val matchData = PrivateMethod[(Boolean, Map[String, Any])]('matchData) -// val result = AccuracyCore invokePrivate matchData(source, target, ruleAnalyzer) -// result._1 should be (true) -// result._2.size should be (0) -// } -// -// test ("match data fail") { -// val rule = "$source.name = $target.name AND $source.age = $target.age" -// val evaluateRuleParam = EvaluateRuleParam(1.0, rule) -// val ruleFactory = RuleFactory(evaluateRuleParam) -// val statement = ruleFactory.generateRule -// val ruleAnalyzer = RuleAnalyzer(statement) -// -// val sourcePersistExprs = ruleAnalyzer.sourceRuleExprs.persistExprs -// val targetPersistExprs = ruleAnalyzer.targetRuleExprs.persistExprs -// -// val source = (Map[String, Any]( -// (findExprId(sourcePersistExprs, "$source['name']") -> "jack"), -// (findExprId(sourcePersistExprs, "$source['age']") -> 26) -// ), Map[String, Any]()) -// val target = (Map[String, Any]( -// (findExprId(targetPersistExprs, "$target['name']") -> "jack"), -// (findExprId(targetPersistExprs, "$target['age']") -> 27) -// ), Map[String, Any]()) -// -// val matchData = PrivateMethod[(Boolean, Map[String, Any])]('matchData) -// val result = AccuracyCore invokePrivate matchData(source, target, ruleAnalyzer) -// result._1 should be (false) -// result._2.size shouldNot be (0) -// } -// -//} diff --git a/measure/src/test/scala/org/apache/griffin/measure/algo/core/ProfileCoreTest.scala b/measure/src/test/scala/org/apache/griffin/measure/algo/core/ProfileCoreTest.scala deleted file mode 100644 index 53b91c044..000000000 --- a/measure/src/test/scala/org/apache/griffin/measure/algo/core/ProfileCoreTest.scala +++ /dev/null @@ -1,79 +0,0 @@ -///* -//Licensed to the Apache Software Foundation (ASF) under one -//or more contributor license agreements. See the NOTICE file -//distributed with this work for additional information -//regarding copyright ownership. The ASF licenses this file -//to you under the Apache License, Version 2.0 (the -//"License"); you may not use this file except in compliance -//with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -//Unless required by applicable law or agreed to in writing, -//software distributed under the License is distributed on an -//"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -//KIND, either express or implied. See the License for the -//specific language governing permissions and limitations -//under the License. -//*/ -//package org.apache.griffin.measure.algo.core -// -//import org.apache.griffin.measure.config.params.user.EvaluateRuleParam -//import org.apache.griffin.measure.rule.expr._ -//import org.apache.griffin.measure.rule.{RuleAnalyzer, RuleFactory} -//import org.junit.runner.RunWith -//import org.scalatest.junit.JUnitRunner -//import org.scalatest.{BeforeAndAfter, FunSuite, Matchers} -//import org.scalatest.PrivateMethodTester -// -//@RunWith(classOf[JUnitRunner]) -//class ProfileCoreTest extends FunSuite with Matchers with BeforeAndAfter with PrivateMethodTester { -// -// def findExprId(exprs: Iterable[Expr], desc: String): String = { -// exprs.find(_.desc == desc) match { -// case Some(expr) => expr._id -// case _ => "" -// } -// } -// -// test ("match data success") { -// val rule = "$source.name = 'jack' AND $source.age = null" -// val evaluateRuleParam = EvaluateRuleParam(1.0, rule) -// val ruleFactory = RuleFactory(evaluateRuleParam) -// val statement = ruleFactory.generateRule -// val ruleAnalyzer = RuleAnalyzer(statement) -// -// val sourcePersistExprs = ruleAnalyzer.sourceRuleExprs.persistExprs -// -// val source = (Map[String, Any]( -// (findExprId(sourcePersistExprs, "$source['name']") -> "jack"), -// (findExprId(sourcePersistExprs, "$source['age']") -> null) -// ), Map[String, Any]()) -// -// val matchData = PrivateMethod[(Boolean, Map[String, Any])]('matchData) -// val result = ProfileCore invokePrivate matchData(source, ruleAnalyzer) -// result._1 should be (true) -// result._2.size should be (0) -// } -// -// test ("match data fail") { -// val rule = "$source.name = 'jack' AND $source.age != null" -// val evaluateRuleParam = EvaluateRuleParam(1.0, rule) -// val ruleFactory = RuleFactory(evaluateRuleParam) -// val statement = ruleFactory.generateRule -// val ruleAnalyzer = RuleAnalyzer(statement) -// -// val sourcePersistExprs = ruleAnalyzer.sourceRuleExprs.persistExprs -// -// val source = (Map[String, Any]( -// (findExprId(sourcePersistExprs, "$source['name']") -> "jack"), -// (findExprId(sourcePersistExprs, "$source['age']") -> null) -// ), Map[String, Any]()) -// -// val matchData = PrivateMethod[(Boolean, Map[String, Any])]('matchData) -// val result = ProfileCore invokePrivate matchData(source, ruleAnalyzer) -// result._1 should be (false) -// result._2.size shouldNot be (0) -// } -// -//} diff --git a/measure/src/test/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgoTest.scala b/measure/src/test/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgoTest.scala deleted file mode 100644 index 9381ab242..000000000 --- a/measure/src/test/scala/org/apache/griffin/measure/algo/streaming/StreamingAccuracyAlgoTest.scala +++ /dev/null @@ -1,261 +0,0 @@ -///* -//Licensed to the Apache Software Foundation (ASF) under one -//or more contributor license agreements. See the NOTICE file -//distributed with this work for additional information -//regarding copyright ownership. The ASF licenses this file -//to you under the Apache License, Version 2.0 (the -//"License"); you may not use this file except in compliance -//with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -//Unless required by applicable law or agreed to in writing, -//software distributed under the License is distributed on an -//"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -//KIND, either express or implied. See the License for the -//specific language governing permissions and limitations -//under the License. -//*/ -//package org.apache.griffin.measure.algo.streaming -// -//import java.util.Date -//import java.util.concurrent.TimeUnit -// -//import org.apache.griffin.measure.algo.batch.BatchAccuracyAlgo -//import org.apache.griffin.measure.cache.info.{InfoCacheInstance, TimeInfoCache} -//import org.apache.griffin.measure.cache.result._ -//import org.apache.griffin.measure.config.params._ -//import org.apache.griffin.measure.config.params.env._ -//import org.apache.griffin.measure.config.params.user._ -//import org.apache.griffin.measure.config.reader._ -//import org.apache.griffin.measure.config.validator._ -//import org.apache.griffin.measure.data.connector.direct.DirectDataConnector -//import org.apache.griffin.measure.data.connector.{DataConnector, DataConnectorFactory} -//import org.apache.griffin.measure.log.Loggable -//import org.apache.griffin.measure.persist.{Persist, PersistFactory, PersistType} -//import org.apache.griffin.measure.result._ -//import org.apache.griffin.measure.rule.expr._ -//import org.apache.griffin.measure.rule.{ExprValueUtil, RuleAnalyzer, RuleFactory} -//import org.apache.griffin.measure.utils.{HdfsUtil, TimeUtil} -//import org.apache.spark.rdd.RDD -//import org.apache.spark.sql.SQLContext -//import org.apache.spark.sql.hive.HiveContext -//import org.apache.spark.streaming.{Milliseconds, StreamingContext} -//import org.apache.spark.{SparkConf, SparkContext} -//import org.junit.runner.RunWith -//import org.scalatest.junit.JUnitRunner -//import org.scalatest.{BeforeAndAfter, FunSuite, Matchers} -// -//import scala.util.{Failure, Success, Try} -// -// -//@RunWith(classOf[JUnitRunner]) -//class StreamingAccuracyAlgoTest extends FunSuite with Matchers with BeforeAndAfter with Loggable { -// -// val envFile = "src/test/resources/env-streaming.json" -// val confFile = "src/test/resources/config-streaming3.json" -// val envFsType = "local" -// val userFsType = "local" -// -// val args = Array(envFile, confFile) -// -// var sc: SparkContext = _ -// var sqlContext: SQLContext = _ -//// val ssc: StreamingContext = _ -// -// var allParam: AllParam = _ -// -// before { -// // read param files -// val envParam = readParamFile[EnvParam](envFile, envFsType) match { -// case Success(p) => p -// case Failure(ex) => { -// error(ex.getMessage) -// sys.exit(-2) -// } -// } -// val userParam = readParamFile[UserParam](confFile, userFsType) match { -// case Success(p) => p -// case Failure(ex) => { -// error(ex.getMessage) -// sys.exit(-2) -// } -// } -// allParam = AllParam(envParam, userParam) -// -// // validate param files -// validateParams(allParam) match { -// case Failure(ex) => { -// error(ex.getMessage) -// sys.exit(-3) -// } -// case _ => { -// info("params validation pass") -// } -// } -// -// val metricName = userParam.name -// val sparkParam = envParam.sparkParam -// val conf = new SparkConf().setMaster("local[*]").setAppName(metricName) -// conf.setAll(sparkParam.config) -// sc = new SparkContext(conf) -// sc.setLogLevel(envParam.sparkParam.logLevel) -// sqlContext = new SQLContext(sc) -//// sqlContext = new HiveContext(sc) -// -//// val a = sqlContext.sql("select * from s1 limit 10") -//// // val a = sqlContext.sql("show tables") -//// a.show(10) -//// -//// val b = HdfsUtil.existPath("/griffin/streaming") -//// println(b) -// } -// -// test("algorithm") { -// val envParam = allParam.envParam -// val userParam = allParam.userParam -// val metricName = userParam.name -// val sparkParam = envParam.sparkParam -// val cleanerParam = envParam.cleanerParam -// -//// val batchInterval = TimeUtil.milliseconds(sparkParam.batchInterval) match { -//// case Some(interval) => Milliseconds(interval) -//// case _ => throw new Exception("invalid batch interval") -//// } -//// val ssc = new StreamingContext(sc, batchInterval) -//// ssc.checkpoint(sparkParam.cpDir) -// -// def createStreamingContext(): StreamingContext = { -// val batchInterval = TimeUtil.milliseconds(sparkParam.batchInterval) match { -// case Some(interval) => Milliseconds(interval) -// case _ => throw new Exception("invalid batch interval") -// } -// val ssc = new StreamingContext(sc, batchInterval) -// ssc.checkpoint(sparkParam.cpDir) -// ssc -// } -// val ssc = StreamingContext.getOrCreate(sparkParam.cpDir, createStreamingContext) -// -// // start time -// val startTime = new Date().getTime() -// -// val persistFactory = PersistFactory(envParam.persistParams, metricName) -// -// // get persists to persist measure result -// val appPersist: Persist = persistFactory.getPersists(startTime) -// -// // get spark application id -// val applicationId = sc.applicationId -// -// // persist start id -// appPersist.start(applicationId) -// -// InfoCacheInstance.initInstance(envParam.infoCacheParams, metricName) -// InfoCacheInstance.init -// -// // generate rule from rule param, generate rule analyzer -// val ruleFactory = RuleFactory(userParam.evaluateRuleParam) -// val rule: StatementExpr = ruleFactory.generateRule() -// val ruleAnalyzer: RuleAnalyzer = RuleAnalyzer(rule) -// -// // const expr value map -// val constExprValueMap = ExprValueUtil.genExprValueMaps(None, ruleAnalyzer.constCacheExprs, Map[String, Any]()) -// val finalConstExprValueMap = ExprValueUtil.updateExprValueMaps(ruleAnalyzer.constFinalCacheExprs, constExprValueMap) -// val finalConstMap = finalConstExprValueMap.headOption match { -// case Some(m) => m -// case _ => Map[String, Any]() -// } -// -// // data connector -// val sourceDataConnector: DirectDataConnector = -// DataConnectorFactory.getDirectDataConnector(sqlContext, ssc, userParam.sourceParam, -// ruleAnalyzer.sourceRuleExprs, finalConstMap -// ) match { -// case Success(cntr) => { -// if (cntr.available) cntr -// else throw new Exception("source data connection error!") -// } -// case Failure(ex) => throw ex -// } -// val targetDataConnector: DirectDataConnector = -// DataConnectorFactory.getDirectDataConnector(sqlContext, ssc, userParam.targetParam, -// ruleAnalyzer.targetRuleExprs, finalConstMap -// ) match { -// case Success(cntr) => { -// if (cntr.available) cntr -// else throw new Exception("target data connection error!") -// } -// case Failure(ex) => throw ex -// } -// -// val cacheResultProcesser = CacheResultProcesser() -// -// // init data stream -// sourceDataConnector.init() -// targetDataConnector.init() -// -// // my algo -// val algo = StreamingAccuracyAlgo(allParam) -// -// val streamingAccuracyProcess = StreamingAccuracyProcess( -// sourceDataConnector, targetDataConnector, -// ruleAnalyzer, cacheResultProcesser, persistFactory, appPersist) -// -// val processInterval = TimeUtil.milliseconds(sparkParam.processInterval) match { -// case Some(interval) => interval -// case _ => throw new Exception("invalid batch interval") -// } -// val process = TimingProcess(processInterval, streamingAccuracyProcess) -// -// // clean thread -//// case class Clean() extends Runnable { -//// val lock = InfoCacheInstance.genLock("clean") -//// def run(): Unit = { -//// val locked = lock.lock(5, TimeUnit.SECONDS) -//// if (locked) { -//// try { -//// sourceDataConnector.cleanData -//// targetDataConnector.cleanData -//// } finally { -//// lock.unlock() -//// } -//// } -//// } -//// } -//// val cleanInterval = TimeUtil.milliseconds(cleanerParam.cleanInterval) match { -//// case Some(interval) => interval -//// case _ => throw new Exception("invalid batch interval") -//// } -//// val clean = TimingProcess(cleanInterval, Clean()) -// -// process.startup() -//// clean.startup() -// -// ssc.start() -// ssc.awaitTermination() -// ssc.stop(stopSparkContext=true, stopGracefully=true) -// -// println("================ end ================") -// -// // context stop -// sc.stop -// -// InfoCacheInstance.close -// -// appPersist.finish() -// -// process.shutdown() -//// clean.shutdown() -// } -// -// private def readParamFile[T <: Param](file: String, fsType: String)(implicit m : Manifest[T]): Try[T] = { -// val paramReader = ParamReaderFactory.getParamReader(file, fsType) -// paramReader.readConfig[T] -// } -// -// private def validateParams(allParam: AllParam): Try[Boolean] = { -// val allParamValidator = AllParamValidator() -// allParamValidator.validate(allParam) -// } -// -//} diff --git a/measure/src/test/scala/org/apache/griffin/measure/data/connector/ConnectorTest.scala b/measure/src/test/scala/org/apache/griffin/measure/data/connector/ConnectorTest.scala index 978863d43..ead84f735 100644 --- a/measure/src/test/scala/org/apache/griffin/measure/data/connector/ConnectorTest.scala +++ b/measure/src/test/scala/org/apache/griffin/measure/data/connector/ConnectorTest.scala @@ -22,18 +22,16 @@ import java.util.Date import java.util.concurrent.TimeUnit import kafka.serializer.StringDecoder -import org.apache.griffin.measure.algo.streaming.TimingProcess import org.apache.griffin.measure.cache.info.InfoCacheInstance import org.apache.griffin.measure.config.params.env._ import org.apache.griffin.measure.config.params.user.{DataConnectorParam, EvaluateRuleParam} import org.apache.griffin.measure.config.reader.ParamRawStringReader import org.apache.griffin.measure.data.connector.batch.TextDirBatchDataConnector +import org.apache.griffin.measure.process.TimingProcess import org.apache.griffin.measure.process.engine.DqEngines import org.apache.griffin.measure.result.{DataInfo, TimeStampInfo} -import org.apache.griffin.measure.rule.expr.{Expr, StatementExpr} import org.apache.griffin.measure.rule._ import org.apache.griffin.measure.utils.{HdfsFileDumpUtil, HdfsUtil, TimeUtil} -import org.apache.griffin.measure.rule.{DataTypeCalculationUtil, ExprValueUtil, RuleExprs} import org.apache.spark.rdd.RDD import org.apache.spark.sql.hive.HiveContext import org.apache.spark.sql.types.{DataType, StructField, StructType} diff --git a/measure/src/test/scala/org/apache/griffin/measure/process/BatchProcessTest.scala b/measure/src/test/scala/org/apache/griffin/measure/process/BatchProcessTest.scala index 3a2b3e84a..41e61c5ad 100644 --- a/measure/src/test/scala/org/apache/griffin/measure/process/BatchProcessTest.scala +++ b/measure/src/test/scala/org/apache/griffin/measure/process/BatchProcessTest.scala @@ -36,8 +36,8 @@ import scala.util.{Failure, Success, Try} class BatchProcessTest extends FunSuite with Matchers with BeforeAndAfter with Loggable { val envFile = "src/test/resources/env-test.json" -// val confFile = "src/test/resources/config-test-profiling.json" - val confFile = "src/test/resources/config-test-accuracy.json" + val confFile = "src/test/resources/config-test-profiling.json" +// val confFile = "src/test/resources/config-test-accuracy.json" val envFsType = "local" val userFsType = "local" diff --git a/measure/src/test/scala/org/apache/griffin/measure/process/StreamingProcessTest.scala b/measure/src/test/scala/org/apache/griffin/measure/process/StreamingProcessTest.scala index 9e6cdf854..a41bb4190 100644 --- a/measure/src/test/scala/org/apache/griffin/measure/process/StreamingProcessTest.scala +++ b/measure/src/test/scala/org/apache/griffin/measure/process/StreamingProcessTest.scala @@ -36,8 +36,8 @@ import scala.util.{Failure, Success, Try} class StreamingProcessTest extends FunSuite with Matchers with BeforeAndAfter with Loggable { val envFile = "src/test/resources/env-streaming.json" - val confFile = "src/test/resources/config-test-accuracy-streaming-multids.json" -// val confFile = "src/test/resources/config-test-accuracy-streaming.json" +// val confFile = "src/test/resources/config-test-accuracy-streaming-multids.json" + val confFile = "src/test/resources/config-test-accuracy-streaming.json" // val confFile = "src/test/resources/config-test-profiling-streaming.json" val envFsType = "local" diff --git a/measure/src/test/scala/org/apache/griffin/measure/rule/ExprValueUtilTest.scala b/measure/src/test/scala/org/apache/griffin/measure/rule/ExprValueUtilTest.scala deleted file mode 100644 index 9cfcab55a..000000000 --- a/measure/src/test/scala/org/apache/griffin/measure/rule/ExprValueUtilTest.scala +++ /dev/null @@ -1,86 +0,0 @@ -///* -//Licensed to the Apache Software Foundation (ASF) under one -//or more contributor license agreements. See the NOTICE file -//distributed with this work for additional information -//regarding copyright ownership. The ASF licenses this file -//to you under the Apache License, Version 2.0 (the -//"License"); you may not use this file except in compliance -//with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -//Unless required by applicable law or agreed to in writing, -//software distributed under the License is distributed on an -//"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -//KIND, either express or implied. See the License for the -//specific language governing permissions and limitations -//under the License. -//*/ -//package org.apache.griffin.measure.rule -// -//import org.apache.griffin.measure.config.params.user.EvaluateRuleParam -//import org.apache.griffin.measure.rule.expr.{Expr, StatementExpr} -//import org.junit.runner.RunWith -//import org.scalatest.junit.JUnitRunner -//import org.scalatest.{BeforeAndAfter, FunSuite, Matchers} -// -//@RunWith(classOf[JUnitRunner]) -//class ExprValueUtilTest extends FunSuite with BeforeAndAfter with Matchers { -// -// test ("rule calculation") { -// // val rules = "$source.json().name = 's2' and $source.json().age[*] = 32" -// // val rules = "$source.json().items[*] = 202 AND $source.json().age[*] = 32 AND $source.json().df[*].a = 1" -// val rules = "$source.json().items[*] = 202 AND $source.json().age[*] = 32 AND $source.json().df['a' = 1].b = 4" -// // val rules = "$source.json().df[0].a = 1" -// val ep = EvaluateRuleParam(1, rules) -// -// val ruleFactory = RuleFactory(ep) -// val rule: StatementExpr = ruleFactory.generateRule() -// val ruleAnalyzer: RuleAnalyzer = RuleAnalyzer(rule) -// -// val ruleExprs = ruleAnalyzer.sourceRuleExprs -// val constFinalExprValueMap = Map[String, Any]() -// -// val data = List[String]( -// ("""{"name": "s1", "age": [22, 23], "items": [102, 104, 106], "df": [{"a": 1, "b": 3}, {"a": 2, "b": 4}]}"""), -// ("""{"name": "s2", "age": [32, 33], "items": [202, 204, 206], "df": [{"a": 1, "b": 4}, {"a": 2, "b": 4}]}"""), -// ("""{"name": "s3", "age": [42, 43], "items": [302, 304, 306], "df": [{"a": 1, "b": 5}, {"a": 2, "b": 4}]}""") -// ) -// -// def str(expr: Expr) = { -// s"${expr._id}: ${expr.desc} [${expr.getClass.getSimpleName}]" -// } -// println("====") -// ruleExprs.finalCacheExprs.foreach { expr => -// println(str(expr)) -// } -// println("====") -// ruleExprs.cacheExprs.foreach { expr => -// println(str(expr)) -// } -// -// val constExprValueMap = ExprValueUtil.genExprValueMaps(None, ruleAnalyzer.constCacheExprs, Map[String, Any]()) -// val finalConstExprValueMap = ExprValueUtil.updateExprValueMaps(ruleAnalyzer.constFinalCacheExprs, constExprValueMap) -// val finalConstMap = finalConstExprValueMap.headOption match { -// case Some(m) => m -// case _ => Map[String, Any]() -// } -// println("====") -// println(ruleAnalyzer.constCacheExprs) -// println(ruleAnalyzer.constFinalCacheExprs) -// println(finalConstMap) -// -// println("====") -// val valueMaps = data.flatMap { msg => -// val cacheExprValueMaps = ExprValueUtil.genExprValueMaps(Some(msg), ruleExprs.cacheExprs, finalConstMap) -// val finalExprValueMaps = ExprValueUtil.updateExprValueMaps(ruleExprs.finalCacheExprs, cacheExprValueMaps) -// -// finalExprValueMaps -// } -// -// valueMaps.foreach(println) -// println(valueMaps.size) -// -// } -// -//} diff --git a/measure/src/test/scala/org/apache/griffin/measure/rule/RuleAnalyzerTest.scala b/measure/src/test/scala/org/apache/griffin/measure/rule/RuleAnalyzerTest.scala deleted file mode 100644 index 0b4c16cfa..000000000 --- a/measure/src/test/scala/org/apache/griffin/measure/rule/RuleAnalyzerTest.scala +++ /dev/null @@ -1,60 +0,0 @@ -///* -//Licensed to the Apache Software Foundation (ASF) under one -//or more contributor license agreements. See the NOTICE file -//distributed with this work for additional information -//regarding copyright ownership. The ASF licenses this file -//to you under the Apache License, Version 2.0 (the -//"License"); you may not use this file except in compliance -//with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -//Unless required by applicable law or agreed to in writing, -//software distributed under the License is distributed on an -//"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -//KIND, either express or implied. See the License for the -//specific language governing permissions and limitations -//under the License. -//*/ -//package org.apache.griffin.measure.rule -// -//import org.apache.griffin.measure.config.params.user.EvaluateRuleParam -//import org.junit.runner.RunWith -//import org.scalatest.junit.JUnitRunner -//import org.scalatest.{BeforeAndAfter, FunSuite, Matchers} -// -//@RunWith(classOf[JUnitRunner]) -//class RuleAnalyzerTest extends FunSuite with BeforeAndAfter with Matchers { -// -// test ("rule analyze") { -// val rule = "$source.name = $target.name AND $source.age = $target.age + (2 * 5) AND $source.born > (6 - 2 * 2)" -// val evaluateRuleParam = EvaluateRuleParam(1.0, rule) -// val ruleFactory = RuleFactory(evaluateRuleParam) -// val statement = ruleFactory.generateRule -// -// val ruleAnalyzer = RuleAnalyzer(statement) -// -// ruleAnalyzer.constCacheExprs.map(_.desc) should be (List[String]("2 * 5", "2 * 2", "6 - 2 * 2")) -// ruleAnalyzer.constFinalCacheExprs.map(_.desc) should be (Set[String]("2 * 5", "6 - 2 * 2")) -// -// ruleAnalyzer.sourceRuleExprs.groupbyExprs.map(_.desc) should be (List[String]( -// "$source['name']", "$source['age']")) -// ruleAnalyzer.sourceRuleExprs.cacheExprs.map(_.desc) should be (List[String]( -// "$source['name']", "$source['age']", "$source['born']", "$source['born'] > 6 - 2 * 2")) -// ruleAnalyzer.sourceRuleExprs.finalCacheExprs.map(_.desc) should be (Set[String]( -// "$source['name']", "$source['age']", "$source['born']", "$source['born'] > 6 - 2 * 2")) -// ruleAnalyzer.sourceRuleExprs.persistExprs.map(_.desc) should be (List[String]( -// "$source['name']", "$source['age']", "$source['born']")) -// -// ruleAnalyzer.targetRuleExprs.groupbyExprs.map(_.desc) should be (List[String]( -// "$target['name']", "$target['age'] + 2 * 5")) -// ruleAnalyzer.targetRuleExprs.cacheExprs.map(_.desc) should be (List[String]( -// "$target['name']", "$target['age']", "$target['age'] + 2 * 5")) -// ruleAnalyzer.targetRuleExprs.finalCacheExprs.map(_.desc) should be (Set[String]( -// "$target['name']", "$target['age']", "$target['age'] + 2 * 5")) -// ruleAnalyzer.targetRuleExprs.persistExprs.map(_.desc) should be (List[String]( -// "$target['name']", "$target['age']")) -// -// } -// -//} diff --git a/measure/src/test/scala/org/apache/griffin/measure/rule/RuleFactoryTest.scala b/measure/src/test/scala/org/apache/griffin/measure/rule/RuleFactoryTest.scala deleted file mode 100644 index 9ed641a0c..000000000 --- a/measure/src/test/scala/org/apache/griffin/measure/rule/RuleFactoryTest.scala +++ /dev/null @@ -1,44 +0,0 @@ -///* -//Licensed to the Apache Software Foundation (ASF) under one -//or more contributor license agreements. See the NOTICE file -//distributed with this work for additional information -//regarding copyright ownership. The ASF licenses this file -//to you under the Apache License, Version 2.0 (the -//"License"); you may not use this file except in compliance -//with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -//Unless required by applicable law or agreed to in writing, -//software distributed under the License is distributed on an -//"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -//KIND, either express or implied. See the License for the -//specific language governing permissions and limitations -//under the License. -//*/ -//package org.apache.griffin.measure.rule -// -//import org.apache.griffin.measure.config.params.user.EvaluateRuleParam -//import org.junit.runner.RunWith -//import org.scalatest.junit.JUnitRunner -//import org.scalatest.{BeforeAndAfter, FunSuite, Matchers} -// -//@RunWith(classOf[JUnitRunner]) -//class RuleFactoryTest extends FunSuite with BeforeAndAfter with Matchers { -// -// test ("generate rule") { -// val rule = "$source.name = $target.name AND $source.age = $target.age" -// val evaluateRuleParam = EvaluateRuleParam(1.0, rule) -// val ruleFactory = RuleFactory(evaluateRuleParam) -// ruleFactory.generateRule.desc should be ("$source['name'] = $target['name'] AND $source['age'] = $target['age']") -// -// val wrong_rule = "$source.name = $target.name AND $source.age = $target1.age" -// val evaluateRuleParam1 = EvaluateRuleParam(1.0, wrong_rule) -// val ruleFactory1 = RuleFactory(evaluateRuleParam1) -// val thrown = intercept[Exception] { -// ruleFactory1.generateRule -// } -// thrown.getMessage should be ("parse rule error!") -// } -// -//} diff --git a/measure/src/test/scala/org/apache/griffin/measure/rule/RuleParserTest.scala b/measure/src/test/scala/org/apache/griffin/measure/rule/RuleParserTest.scala deleted file mode 100644 index 15f1a564c..000000000 --- a/measure/src/test/scala/org/apache/griffin/measure/rule/RuleParserTest.scala +++ /dev/null @@ -1,213 +0,0 @@ -///* -//Licensed to the Apache Software Foundation (ASF) under one -//or more contributor license agreements. See the NOTICE file -//distributed with this work for additional information -//regarding copyright ownership. The ASF licenses this file -//to you under the Apache License, Version 2.0 (the -//"License"); you may not use this file except in compliance -//with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -//Unless required by applicable law or agreed to in writing, -//software distributed under the License is distributed on an -//"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -//KIND, either express or implied. See the License for the -//specific language governing permissions and limitations -//under the License. -//*/ -//package org.apache.griffin.measure.rule -// -//import org.junit.runner.RunWith -//import org.scalatest.junit.JUnitRunner -//import org.scalatest.{BeforeAndAfter, FunSuite, Matchers} -////import org.scalatest.FlatSpec -////import org.scalamock.scalatest.MockFactory -// -//@RunWith(classOf[JUnitRunner]) -//class RuleParserTest extends FunSuite with Matchers with BeforeAndAfter { -// -// val ruleParser = RuleParser() -// -// test ("literal number") { -// val rule1 = "123" -// val result1 = ruleParser.parseAll(ruleParser.literal, rule1) -// result1.successful should be (true) -// result1.get.value should be (Some(123)) -// -// val rule2 = "12.3" -// val result2 = ruleParser.parseAll(ruleParser.literal, rule2) -// result2.successful should be (true) -// result2.get.value should be (Some(12.3)) -// } -// -// test ("literial string") { -// val rule1 = "'123'" -// val result1 = ruleParser.parseAll(ruleParser.literal, rule1) -// result1.successful should be (true) -// result1.get.value should be (Some("123")) -// -// val rule2 = "\"123\"" -// val result2 = ruleParser.parseAll(ruleParser.literal, rule1) -// result2.successful should be (true) -// result2.get.value should be (Some("123")) -// -// val rule3 = "'1+2-3'" -// val result3 = ruleParser.parseAll(ruleParser.literal, rule3) -// result3.successful should be (true) -// result3.get.value should be (Some("1+2-3")) -// } -// -// test ("literial time") { -// val rule = "3h" -// val result = ruleParser.parseAll(ruleParser.literal, rule) -// result.successful should be (true) -// result.get.value should be (Some(3*3600*1000)) -// } -// -// test ("literial boolean") { -// val rule = "true" -// val result = ruleParser.parseAll(ruleParser.literal, rule) -// result.successful should be (true) -// result.get.value should be (Some(true)) -// } -// -// test ("literial null") { -// val rule = "null" -// val result = ruleParser.parseAll(ruleParser.literal, rule) -// result.successful should be (true) -// result.get.value should be (Some(null)) -// } -// -// test ("literial none") { -// val rule = "none" -// val result = ruleParser.parseAll(ruleParser.literal, rule) -// result.successful should be (true) -// result.get.value should be (None) -// } -// -// test ("selection head") { -// val rule = "$source" -// val result = ruleParser.parseAll(ruleParser.selectionHead, rule) -// result.successful should be (true) -// result.get.head should be ("source") -// } -// -// test ("field select") { -// val rule = ".name" -// val result = ruleParser.parseAll(ruleParser.selector, rule) -// result.successful should be (true) -// result.get.desc should be ("['name']") -// } -// -// test ("function operation") { -// val rule = ".func(1, 'abc', 3 + 4)" -// val result = ruleParser.parseAll(ruleParser.selector, rule) -// result.successful should be (true) -// result.get.desc should be (".func(1, 'abc', 3 + 4)") -// } -// -// test ("index field range select") { -// val rule1 = "['field']" -// val result1 = ruleParser.parseAll(ruleParser.selector, rule1) -// result1.successful should be (true) -// result1.get.desc should be ("['field']") -// -// val rule2 = "[1, 4]" -// val result2 = ruleParser.parseAll(ruleParser.selector, rule2) -// result2.successful should be (true) -// result2.get.desc should be ("[1, 4]") -// -// val rule3 = "[1, 'name', 'age', 5, (6, 8)]" -// val result3 = ruleParser.parseAll(ruleParser.selector, rule3) -// result3.successful should be (true) -// result3.get.desc should be ("[1, 'name', 'age', 5, (6, 8)]") -// } -// -// test ("index field range") { -// val rule1 = "(3, 5)" -// val result1 = ruleParser.parseAll(ruleParser.indexFieldRange, rule1) -// result1.successful should be (true) -// result1.get.desc should be ("(3, 5)") -// -// val rule2 = "'name'" -// val result2 = ruleParser.parseAll(ruleParser.indexFieldRange, rule2) -// result2.successful should be (true) -// result2.get.desc should be ("'name'") -// -// val rule3 = "*" -// val result3 = ruleParser.parseAll(ruleParser.indexFieldRange, rule3) -// result3.successful should be (true) -// result3.get.desc should be ("*") -// } -// -// test ("filter select") { -// val rule = "['age' > 16]" -// val result = ruleParser.parseAll(ruleParser.selector, rule) -// result.successful should be (true) -// result.get.desc should be ("['age' > 16]") -// } -// -// test ("selection") { -// val rule = "$source['age' > 16].func(1, 'abc')[1, 3, 'name'].time[*]" -// val result = ruleParser.parseAll(ruleParser.selection, rule) -// result.successful should be (true) -// result.get.desc should be ("$source['age' > 16].func(1, 'abc')[1, 3, 'name']['time'][*]") -// } -// -// test ("math expr") { -// val rule = "$source.age * 6 + 4 / 2" -// val result = ruleParser.parseAll(ruleParser.mathExpr, rule) -// result.successful should be (true) -// result.get.desc should be ("$source['age'] * 6 + 4 / 2") -// -// val rule2 = "'age + 1' / 'vv'" -// val result2 = ruleParser.parseAll(ruleParser.mathExpr, rule2) -// result2.successful should be (true) -// result2.get.desc should be ("'age + 1' / 'vv'") -// println(result2) -// } -// -// test ("range expr") { -// val rule = "($source.age + 1, $target.age + 3, 40)" -// val result = ruleParser.parseAll(ruleParser.rangeExpr, rule) -// result.successful should be (true) -// result.get.desc should be ("($source['age'] + 1, $target['age'] + 3, 40)") -// } -// -// test ("logical expr") { -// val rule1 = "$source.age + 1 = $target.age" -// val result1 = ruleParser.parseAll(ruleParser.logicalExpr, rule1) -// result1.successful should be (true) -// result1.get.desc should be ("$source['age'] + 1 = $target['age']") -// -// val rule2 = "$source.age in (3, 5, 6, 10)" -// val result2 = ruleParser.parseAll(ruleParser.logicalExpr, rule2) -// result2.successful should be (true) -// result2.get.desc should be ("$source['age'] in (3, 5, 6, 10)") -// } -// -// test ("logical statement") { -// val rule1 = "$source.descs[0] = $target.desc AND $source.name = $target.name" -// val result1 = ruleParser.parseAll(ruleParser.logicalStatement, rule1) -// result1.successful should be (true) -// result1.get.desc should be ("$source['descs'][0] = $target['desc'] AND $source['name'] = $target['name']") -// -// val rule2 = "NOT $source.age = $target.age" -// val result2 = ruleParser.parseAll(ruleParser.logicalStatement, rule2) -// result2.successful should be (true) -// result2.get.desc should be ("NOT $source['age'] = $target['age']") -// } -// -// test ("whole rule") { -// val rule1 = "$source.name = $target.name AND $source.age = $target.age" -// val result1 = ruleParser.parseAll(ruleParser.rule, rule1) -// result1.successful should be (true) -// result1.get.desc should be ("$source['name'] = $target['name'] AND $source['age'] = $target['age']") -// -// val rule2 = "$source.name = $target.name AND $source.age = $target.age AND $source.id > 1000" -// val result2 = ruleParser.parseAll(ruleParser.rule, rule2) -// result2.successful should be (true) -// result2.get.desc should be ("$source['name'] = $target['name'] AND $source['age'] = $target['age'] AND $source['id'] > 1000") -// } -//} diff --git a/measure/src/test/scala/org/apache/griffin/measure/rules/dsl/parser/BasicParserTest.scala b/measure/src/test/scala/org/apache/griffin/measure/rule/dsl/parser/BasicParserTest.scala similarity index 98% rename from measure/src/test/scala/org/apache/griffin/measure/rules/dsl/parser/BasicParserTest.scala rename to measure/src/test/scala/org/apache/griffin/measure/rule/dsl/parser/BasicParserTest.scala index 1d267985f..f70d77186 100644 --- a/measure/src/test/scala/org/apache/griffin/measure/rules/dsl/parser/BasicParserTest.scala +++ b/measure/src/test/scala/org/apache/griffin/measure/rule/dsl/parser/BasicParserTest.scala @@ -16,9 +16,9 @@ KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License. */ -package org.apache.griffin.measure.rules.dsl.parser +package org.apache.griffin.measure.rule.dsl.parser -import org.apache.griffin.measure.rules.dsl.expr._ +import org.apache.griffin.measure.rule.dsl.expr._ import org.junit.runner.RunWith import org.scalatest.junit.JUnitRunner import org.scalatest.{BeforeAndAfter, FunSuite, Matchers} From 328a9bc82de931ca9cca244e6fc6e11f4cd0a0e2 Mon Sep 17 00:00:00 2001 From: Lionel Liu Date: Mon, 25 Sep 2017 15:42:06 +0800 Subject: [PATCH 080/111] crawler-test --- .../main/scala/org/apache/griffin/measure/Application.scala | 3 --- .../apache/griffin/measure/data/connector/DataConnector.scala | 1 - .../griffin/measure/data/connector/DataConnectorFactory.scala | 1 - .../measure/data/connector/batch/AvroBatchDataConnector.scala | 1 - .../measure/data/connector/batch/HiveBatchDataConnector.scala | 1 - .../org/apache/griffin/measure/process/BatchDqProcess.scala | 1 - .../org/apache/griffin/measure/process/StreamingDqThread.scala | 1 - .../org/apache/griffin/measure/rule/adaptor/RuleAdaptor.scala | 1 - .../apache/griffin/measure/rule/adaptor/SparkSqlAdaptor.scala | 1 - .../main/scala/org/apache/griffin/measure/utils/HdfsUtil.scala | 2 +- .../org/apache/griffin/measure/process/BatchProcessTest.scala | 1 - .../org/apache/griffin/measure/process/JsonParseTest.scala | 1 - .../apache/griffin/measure/process/StreamingProcessTest.scala | 1 - 13 files changed, 1 insertion(+), 15 deletions(-) diff --git a/measure/src/main/scala/org/apache/griffin/measure/Application.scala b/measure/src/main/scala/org/apache/griffin/measure/Application.scala index dd5742bad..edbb552af 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/Application.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/Application.scala @@ -18,9 +18,6 @@ under the License. */ package org.apache.griffin.measure -import org.apache.griffin.measure.algo._ -//import org.apache.griffin.measure.algo.batch._ -//import org.apache.griffin.measure.algo.streaming._ import org.apache.griffin.measure.config.params._ import org.apache.griffin.measure.config.params.env._ import org.apache.griffin.measure.config.params.user._ diff --git a/measure/src/main/scala/org/apache/griffin/measure/data/connector/DataConnector.scala b/measure/src/main/scala/org/apache/griffin/measure/data/connector/DataConnector.scala index a4fb56bac..52d4fcd37 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/data/connector/DataConnector.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/data/connector/DataConnector.scala @@ -20,7 +20,6 @@ package org.apache.griffin.measure.data.connector import java.util.concurrent.atomic.AtomicLong -import org.apache.griffin.measure.algo._ import org.apache.griffin.measure.config.params.user.DataConnectorParam import org.apache.griffin.measure.log.Loggable import org.apache.griffin.measure.process.engine._ diff --git a/measure/src/main/scala/org/apache/griffin/measure/data/connector/DataConnectorFactory.scala b/measure/src/main/scala/org/apache/griffin/measure/data/connector/DataConnectorFactory.scala index 9d190bbd9..9c3383f67 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/data/connector/DataConnectorFactory.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/data/connector/DataConnectorFactory.scala @@ -29,7 +29,6 @@ import scala.util.Success //import org.apache.griffin.measure.data.connector.cache._ import org.apache.griffin.measure.data.connector.batch._ //import org.apache.griffin.measure.data.connector.streaming._ -import org.apache.griffin.measure.rule.RuleExprs import org.apache.spark.sql.SQLContext import org.apache.spark.streaming.StreamingContext import org.apache.spark.streaming.dstream.InputDStream diff --git a/measure/src/main/scala/org/apache/griffin/measure/data/connector/batch/AvroBatchDataConnector.scala b/measure/src/main/scala/org/apache/griffin/measure/data/connector/batch/AvroBatchDataConnector.scala index 3d0fcfdfd..ccd644198 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/data/connector/batch/AvroBatchDataConnector.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/data/connector/batch/AvroBatchDataConnector.scala @@ -22,7 +22,6 @@ import org.apache.griffin.measure.config.params.user.DataConnectorParam import org.apache.griffin.measure.data.connector._ import org.apache.griffin.measure.process.engine.DqEngines import org.apache.griffin.measure.result._ -import org.apache.griffin.measure.rule.{ExprValueUtil, RuleExprs} import org.apache.griffin.measure.utils.HdfsUtil import org.apache.spark.rdd.RDD import org.apache.spark.sql.{DataFrame, SQLContext} diff --git a/measure/src/main/scala/org/apache/griffin/measure/data/connector/batch/HiveBatchDataConnector.scala b/measure/src/main/scala/org/apache/griffin/measure/data/connector/batch/HiveBatchDataConnector.scala index 87fa4f94a..5d80d0e5f 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/data/connector/batch/HiveBatchDataConnector.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/data/connector/batch/HiveBatchDataConnector.scala @@ -22,7 +22,6 @@ import org.apache.griffin.measure.config.params.user.DataConnectorParam import org.apache.griffin.measure.data.connector._ import org.apache.griffin.measure.process.engine.DqEngines import org.apache.griffin.measure.result._ -import org.apache.griffin.measure.rule.{ExprValueUtil, RuleExprs} import org.apache.spark.rdd.RDD import org.apache.spark.sql.hive.HiveContext import org.apache.spark.sql.{DataFrame, SQLContext} diff --git a/measure/src/main/scala/org/apache/griffin/measure/process/BatchDqProcess.scala b/measure/src/main/scala/org/apache/griffin/measure/process/BatchDqProcess.scala index 029f6a91a..e52ec25ea 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/process/BatchDqProcess.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/process/BatchDqProcess.scala @@ -20,7 +20,6 @@ package org.apache.griffin.measure.process import java.util.Date -import org.apache.griffin.measure.algo._ import org.apache.griffin.measure.config.params._ import org.apache.griffin.measure.config.params.env._ import org.apache.griffin.measure.config.params.user._ diff --git a/measure/src/main/scala/org/apache/griffin/measure/process/StreamingDqThread.scala b/measure/src/main/scala/org/apache/griffin/measure/process/StreamingDqThread.scala index c631b726d..e3b7f8995 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/process/StreamingDqThread.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/process/StreamingDqThread.scala @@ -21,7 +21,6 @@ package org.apache.griffin.measure.process import java.util.Date import java.util.concurrent.TimeUnit -import org.apache.griffin.measure.algo._ import org.apache.griffin.measure.cache.info.{InfoCacheInstance, TimeInfoCache} import org.apache.griffin.measure.cache.result.CacheResultProcesser import org.apache.griffin.measure.config.params.user.EvaluateRuleParam diff --git a/measure/src/main/scala/org/apache/griffin/measure/rule/adaptor/RuleAdaptor.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/adaptor/RuleAdaptor.scala index 231722581..744f52ab0 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rule/adaptor/RuleAdaptor.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rule/adaptor/RuleAdaptor.scala @@ -20,7 +20,6 @@ package org.apache.griffin.measure.rule.adaptor import java.util.concurrent.atomic.AtomicLong -import org.apache.griffin.measure.algo._ import scala.collection.mutable.{Set => MutableSet} import org.apache.griffin.measure.config.params.user._ diff --git a/measure/src/main/scala/org/apache/griffin/measure/rule/adaptor/SparkSqlAdaptor.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/adaptor/SparkSqlAdaptor.scala index 7d5a24cae..78121fa00 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rule/adaptor/SparkSqlAdaptor.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rule/adaptor/SparkSqlAdaptor.scala @@ -18,7 +18,6 @@ under the License. */ package org.apache.griffin.measure.rule.adaptor -import org.apache.griffin.measure.algo._ import org.apache.griffin.measure.data.connector.GroupByColumn import org.apache.griffin.measure.rule.step._ diff --git a/measure/src/main/scala/org/apache/griffin/measure/utils/HdfsUtil.scala b/measure/src/main/scala/org/apache/griffin/measure/utils/HdfsUtil.scala index 69f63beed..9fa6bcfbe 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/utils/HdfsUtil.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/utils/HdfsUtil.scala @@ -28,7 +28,7 @@ object HdfsUtil extends Loggable { private val conf = new Configuration() conf.set("dfs.support.append", "true") - conf.set("fs.defaultFS", "hdfs://localhost") // debug @localhost +// conf.set("fs.defaultFS", "hdfs://localhost") // debug @localhost private val dfs = FileSystem.get(conf) diff --git a/measure/src/test/scala/org/apache/griffin/measure/process/BatchProcessTest.scala b/measure/src/test/scala/org/apache/griffin/measure/process/BatchProcessTest.scala index 41e61c5ad..a1e4854b4 100644 --- a/measure/src/test/scala/org/apache/griffin/measure/process/BatchProcessTest.scala +++ b/measure/src/test/scala/org/apache/griffin/measure/process/BatchProcessTest.scala @@ -18,7 +18,6 @@ under the License. */ package org.apache.griffin.measure.process -import org.apache.griffin.measure.algo._ import org.apache.griffin.measure.config.params.env._ import org.apache.griffin.measure.config.params.user._ import org.apache.griffin.measure.config.params._ diff --git a/measure/src/test/scala/org/apache/griffin/measure/process/JsonParseTest.scala b/measure/src/test/scala/org/apache/griffin/measure/process/JsonParseTest.scala index 0d72ae1ee..b119d7650 100644 --- a/measure/src/test/scala/org/apache/griffin/measure/process/JsonParseTest.scala +++ b/measure/src/test/scala/org/apache/griffin/measure/process/JsonParseTest.scala @@ -18,7 +18,6 @@ under the License. */ package org.apache.griffin.measure.process -import org.apache.griffin.measure.algo._ import org.apache.griffin.measure.config.params._ import org.apache.griffin.measure.config.params.env._ import org.apache.griffin.measure.config.params.user._ diff --git a/measure/src/test/scala/org/apache/griffin/measure/process/StreamingProcessTest.scala b/measure/src/test/scala/org/apache/griffin/measure/process/StreamingProcessTest.scala index a41bb4190..07b7c5e08 100644 --- a/measure/src/test/scala/org/apache/griffin/measure/process/StreamingProcessTest.scala +++ b/measure/src/test/scala/org/apache/griffin/measure/process/StreamingProcessTest.scala @@ -18,7 +18,6 @@ under the License. */ package org.apache.griffin.measure.process -import org.apache.griffin.measure.algo._ import org.apache.griffin.measure.config.params._ import org.apache.griffin.measure.config.params.env._ import org.apache.griffin.measure.config.params.user._ From 1baa2124f0096fd0db3c6abc2cdebb6146a65d99 Mon Sep 17 00:00:00 2001 From: Lionel Liu Date: Mon, 25 Sep 2017 23:39:26 +0800 Subject: [PATCH 081/111] optimized --- .../measure/data/source/DataSource.scala | 4 + .../measure/data/source/DataSourceCache.scala | 65 +++++++++++++ .../griffin/measure/persist/HdfsPersist.scala | 24 +++++ .../griffin/measure/persist/HttpPersist.scala | 1 + .../measure/persist/LoggerPersist.scala | 12 +++ .../measure/persist/MultiPersists.scala | 1 + .../griffin/measure/persist/Persist.scala | 1 + .../measure/process/BatchDqProcess.scala | 6 +- .../measure/process/StreamingDqThread.scala | 22 ++++- .../measure/process/engine/DqEngine.scala | 2 + .../measure/process/engine/DqEngines.scala | 92 ++++++++++++++----- .../process/engine/SparkDqEngine.scala | 30 ++++++ .../measure/process/BatchProcessTest.scala | 4 +- 13 files changed, 233 insertions(+), 31 deletions(-) diff --git a/measure/src/main/scala/org/apache/griffin/measure/data/source/DataSource.scala b/measure/src/main/scala/org/apache/griffin/measure/data/source/DataSource.scala index 8e0f73143..0927754ae 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/data/source/DataSource.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/data/source/DataSource.scala @@ -98,6 +98,10 @@ case class DataSource(sqlContext: SQLContext, dataSourceCacheOpt.foreach(_.updateData(df, ms)) } + def updateDataMap(dfMap: Map[Long, DataFrame]): Unit = { + dataSourceCacheOpt.foreach(_.updateDataMap(dfMap)) + } + def cleanOldData(): Unit = { dataSourceCacheOpt.foreach(_.cleanOldData) } diff --git a/measure/src/main/scala/org/apache/griffin/measure/data/source/DataSourceCache.scala b/measure/src/main/scala/org/apache/griffin/measure/data/source/DataSourceCache.scala index 64877ad97..fc798e6cf 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/data/source/DataSourceCache.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/data/source/DataSourceCache.scala @@ -139,6 +139,7 @@ case class DataSourceCache(sqlContext: SQLContext, param: Map[String, Any], } } + // -- deprecated -- def updateData(df: DataFrame, ms: Long): Unit = { val ptns = getPartition(ms) val ptnsPath = genPartitionHdfsPath(ptns) @@ -165,6 +166,70 @@ case class DataSourceCache(sqlContext: SQLContext, param: Map[String, Any], } } + def updateData(rdd: RDD[String], ms: Long, cnt: Long): Unit = { + val ptns = getPartition(ms) + val ptnsPath = genPartitionHdfsPath(ptns) + val dirPath = s"${filePath}/${ptnsPath}" + val dataFileName = s"${ms}" + val dataFilePath = HdfsUtil.getHdfsFilePath(dirPath, dataFileName) + + try { +// val needSave = !rdd.isEmpty + + // remove out time old data + HdfsFileDumpUtil.remove(dirPath, dataFileName, true) + println(s"remove file path: ${dirPath}/${dataFileName}") + + // save updated data + val dumped = if (cnt > 0) { + HdfsFileDumpUtil.dump(dataFilePath, rdd, rowSepLiteral) + println(s"update file path: ${dataFilePath}") + } else false + } catch { + case e: Throwable => error(s"update data error: ${e.getMessage}") + } finally { + rdd.unpersist() + } + } + + def updateData(rdd: Iterable[String], ms: Long): Unit = { + val ptns = getPartition(ms) + val ptnsPath = genPartitionHdfsPath(ptns) + val dirPath = s"${filePath}/${ptnsPath}" + val dataFileName = s"${ms}" + val dataFilePath = HdfsUtil.getHdfsFilePath(dirPath, dataFileName) + + try { + val needSave = !rdd.isEmpty + + // remove out time old data + HdfsFileDumpUtil.remove(dirPath, dataFileName, true) + println(s"remove file path: ${dirPath}/${dataFileName}") + + // save updated data + val dumped = if (needSave) { + HdfsFileDumpUtil.dump(dataFilePath, rdd, rowSepLiteral) + println(s"update file path: ${dataFilePath}") + } else false + } catch { + case e: Throwable => error(s"update data error: ${e.getMessage}") + } + } + + def updateDataMap(dfMap: Map[Long, DataFrame]): Unit = { + val dataMap = dfMap.map { pair => + val (t, recs) = pair + val rdd = recs.toJSON +// rdd.cache + (t, rdd, rdd.count) + } + + dataMap.foreach { pair => + val (t, arr, cnt) = pair + updateData(arr, t, cnt) + } + } + def cleanOldData(): Unit = { val oldCacheLocked = oldCacheLock.lock(-1, TimeUnit.SECONDS) if (oldCacheLocked) { diff --git a/measure/src/main/scala/org/apache/griffin/measure/persist/HdfsPersist.scala b/measure/src/main/scala/org/apache/griffin/measure/persist/HdfsPersist.scala index 97cebc92e..d6f927319 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/persist/HdfsPersist.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/persist/HdfsPersist.scala @@ -227,6 +227,30 @@ case class HdfsPersist(config: Map[String, Any], metricName: String, timeStamp: } } + def persistRecords(records: Iterable[String], name: String): Unit = { + val path = filePath(name) + try { + val recordCount = records.size + val count = if (maxPersistLines < 0) recordCount else scala.math.min(maxPersistLines, recordCount) + if (count > 0) { + val groupCount = ((count - 1) / maxLinesPerFile + 1).toInt + if (groupCount <= 1) { + val recs = records.take(count.toInt) + persistRecords(path, recs) + } else { + val groupedRecords = records.grouped(groupCount).zipWithIndex + groupedRecords.take(groupCount).foreach { group => + val (recs, gid) = group + val hdfsPath = if (gid == 0) path else withSuffix(path, gid.toString) + persistRecords(hdfsPath, recs) + } + } + } + } catch { + case e: Throwable => error(e.getMessage) + } + } + // def persistMetrics(metrics: Seq[String], name: String): Unit = { // val path = filePath(name) // try { diff --git a/measure/src/main/scala/org/apache/griffin/measure/persist/HttpPersist.scala b/measure/src/main/scala/org/apache/griffin/measure/persist/HttpPersist.scala index 3a4fa67a1..3c07a9094 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/persist/HttpPersist.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/persist/HttpPersist.scala @@ -90,6 +90,7 @@ case class HttpPersist(config: Map[String, Any], metricName: String, timeStamp: def log(rt: Long, msg: String): Unit = {} def persistRecords(df: DataFrame, name: String): Unit = {} + def persistRecords(records: Iterable[String], name: String): Unit = {} // def persistMetrics(metrics: Seq[String], name: String): Unit = { // val maps = metrics.flatMap { m => diff --git a/measure/src/main/scala/org/apache/griffin/measure/persist/LoggerPersist.scala b/measure/src/main/scala/org/apache/griffin/measure/persist/LoggerPersist.scala index 12672822b..fc565f878 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/persist/LoggerPersist.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/persist/LoggerPersist.scala @@ -132,6 +132,18 @@ case class LoggerPersist(config: Map[String, Any], metricName: String, timeStamp } } + def persistRecords(records: Iterable[String], name: String): Unit = { + try { + val recordCount = records.size + val count = if (maxLogLines < 0) recordCount else scala.math.min(maxLogLines, recordCount) + if (count > 0) { + records.foreach(println) + } + } catch { + case e: Throwable => error(e.getMessage) + } + } + // def persistMetrics(metrics: Seq[String], name: String): Unit = { // try { // val recordCount = metrics.size diff --git a/measure/src/main/scala/org/apache/griffin/measure/persist/MultiPersists.scala b/measure/src/main/scala/org/apache/griffin/measure/persist/MultiPersists.scala index f14f9cbc5..0b8b2f61e 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/persist/MultiPersists.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/persist/MultiPersists.scala @@ -51,6 +51,7 @@ case class MultiPersists(persists: Iterable[Persist]) extends Persist { def log(rt: Long, msg: String): Unit = { persists.foreach(_.log(rt, msg)) } def persistRecords(df: DataFrame, name: String): Unit = { persists.foreach(_.persistRecords(df, name)) } + def persistRecords(records: Iterable[String], name: String): Unit = { persists.foreach(_.persistRecords(records, name)) } // def persistMetrics(metrics: Seq[String], name: String): Unit = { persists.foreach(_.persistMetrics(metrics, name)) } def persistMetrics(metrics: Map[String, Any]): Unit = { persists.foreach(_.persistMetrics(metrics)) } diff --git a/measure/src/main/scala/org/apache/griffin/measure/persist/Persist.scala b/measure/src/main/scala/org/apache/griffin/measure/persist/Persist.scala index 4c73a2b79..d354a5114 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/persist/Persist.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/persist/Persist.scala @@ -44,6 +44,7 @@ trait Persist extends Loggable with Serializable { // def records(recs: Iterable[String], tp: String): Unit def persistRecords(df: DataFrame, name: String): Unit + def persistRecords(records: Iterable[String], name: String): Unit // def persistMetrics(metrics: Seq[String], name: String): Unit def persistMetrics(metrics: Map[String, Any]): Unit diff --git a/measure/src/main/scala/org/apache/griffin/measure/process/BatchDqProcess.scala b/measure/src/main/scala/org/apache/griffin/measure/process/BatchDqProcess.scala index e52ec25ea..a0a2239ef 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/process/BatchDqProcess.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/process/BatchDqProcess.scala @@ -93,7 +93,11 @@ case class BatchDqProcess(allParam: AllParam) extends DqProcess { // persist results val timeGroups = dqEngines.persistAllMetrics(ruleSteps, persistFactory) - dqEngines.persistAllRecords(ruleSteps, persistFactory, timeGroups) + + val stepRdds = dqEngines.collectUpdateRDDs(ruleSteps, timeGroups) + + dqEngines.persistAllRecords(stepRdds, persistFactory) +// dqEngines.persistAllRecords(ruleSteps, persistFactory, timeGroups) // end time val endTime = new Date().getTime diff --git a/measure/src/main/scala/org/apache/griffin/measure/process/StreamingDqThread.scala b/measure/src/main/scala/org/apache/griffin/measure/process/StreamingDqThread.scala index e3b7f8995..c7224c36d 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/process/StreamingDqThread.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/process/StreamingDqThread.scala @@ -62,14 +62,26 @@ case class StreamingDqThread(dqEngines: DqEngines, dqEngines.runRuleSteps(ruleSteps) val ct = new Date().getTime - appPersist.log(ct, s"calculation using time: ${ct - st} ms") + val calculationTimeStr = s"calculation using time: ${ct - st} ms" + println(calculationTimeStr) + appPersist.log(ct, calculationTimeStr) // persist results val timeGroups = dqEngines.persistAllMetrics(ruleSteps, persistFactory) - dqEngines.persistAllRecords(ruleSteps, persistFactory, timeGroups) + + // get rdd of steps + val stepRdds = dqEngines.collectUpdateRDDs(ruleSteps, timeGroups) + + dqEngines.persistAllRecords(stepRdds, persistFactory) + + val rt = new Date().getTime + val persistResultTimeStr = s"persist result and records using time: ${rt - ct} ms" + println(persistResultTimeStr) + appPersist.log(rt, persistResultTimeStr) // update data source - dqEngines.updateDataSources(ruleSteps, dataSources, timeGroups) +// dqEngines.updateDataSources(ruleSteps, dataSources, timeGroups) + dqEngines.updateDataSources(stepRdds, dataSources) TimeInfoCache.endTimeInfoCache @@ -77,7 +89,9 @@ case class StreamingDqThread(dqEngines: DqEngines, cleanData val et = new Date().getTime - appPersist.log(et, s"persist using time: ${et - ct} ms") + val persistTimeStr = s"update data source using time: ${et - rt} ms" + println(persistTimeStr) + appPersist.log(et, persistTimeStr) } catch { case e: Throwable => error(s"process error: ${e.getMessage}") diff --git a/measure/src/main/scala/org/apache/griffin/measure/process/engine/DqEngine.scala b/measure/src/main/scala/org/apache/griffin/measure/process/engine/DqEngine.scala index 9510309d1..c82d318b9 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/process/engine/DqEngine.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/process/engine/DqEngine.scala @@ -36,4 +36,6 @@ trait DqEngine extends Loggable with Serializable { def collectRecords(ruleStep: ConcreteRuleStep, timeGroups: Iterable[Long]): Map[Long, DataFrame] def collectUpdateCacheDatas(ruleStep: ConcreteRuleStep, timeGroups: Iterable[Long]): Map[Long, DataFrame] + + def collectUpdateRDD(ruleStep: ConcreteRuleStep, timeGroups: Iterable[Long]): Option[RDD[(Long, Iterable[String])]] } diff --git a/measure/src/main/scala/org/apache/griffin/measure/process/engine/DqEngines.scala b/measure/src/main/scala/org/apache/griffin/measure/process/engine/DqEngines.scala index 9f581a5ab..d3b4a2b78 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/process/engine/DqEngines.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/process/engine/DqEngines.scala @@ -100,13 +100,22 @@ case class DqEngines(engines: Seq[DqEngine]) extends DqEngine { } } if (udpateDataSources.size > 0) { - val records = collectUpdateCacheDatas(step, timeGroups) +// val records = collectUpdateCacheDatas(step, timeGroups) +// +// udpateDataSources.foreach { ds => +// ds.updateDataMap(records) +// } - records.foreach { pair => - val (t, recs) = pair - udpateDataSources.foreach { ds => - ds.updateData(recs, t) + collectUpdateRDD(step, timeGroups) match { + case Some(rdd) => { + rdd.foreach { pair => + val (t, items) = pair + udpateDataSources.foreach { ds => + ds.dataSourceCacheOpt.foreach(_.updateData(items, t)) + } + } } + case _ => println("empty update data source") } } } @@ -190,26 +199,61 @@ case class DqEngines(engines: Seq[DqEngine]) extends DqEngine { } // if (ret.isEmpty) warn(s"collect metrics warn: no metrics collected for ${ruleStep}") ret -// val ret = engines.foldLeft(Map[Long, Map[String, Any]]()) { (ret, engine) => -// val metrics: Map[Long, Map[String, Any]] = engine.collectMetrics(ruleStep) -// metrics.foldLeft(ret) { (total, pair) => -// val (k, v) = pair -// ret.get(k) match { -// case Some(map) => ret + (k -> (map ++ v)) -// case _ => ret + pair -// } -// } -// } -// if (ret.isEmpty) error(s"collect metrics warn: no metrics collected for ${ruleStep}") -// ret } -// def persistResults(ruleSteps: Seq[ConcreteRuleStep], persist: Persist, persistType: PersistType): Boolean = { -// val ret = engines.foldLeft(false) { (done, engine) => -// done || engine.persistResults(ruleSteps, persist, persistType) -// } -// if (!ret) error(s"persist result warn: no dq engine support ${ruleSteps}") -// ret -// } + def collectUpdateRDD(ruleStep: ConcreteRuleStep, timeGroups: Iterable[Long]): Option[RDD[(Long, Iterable[String])]] = { + engines.flatMap { engine => + engine.collectUpdateRDD(ruleStep, timeGroups) + }.headOption + } + + def collectUpdateRDDs(ruleSteps: Seq[ConcreteRuleStep], timeGroups: Iterable[Long] + ): Seq[(ConcreteRuleStep, RDD[(Long, Iterable[String])])] = { + ruleSteps.flatMap { rs => + collectUpdateRDD(rs, timeGroups) match { + case Some(rdd) => Some((rs, rdd)) + case _ => None + } + } + } + + def persistAllRecords(stepRdds: Seq[(ConcreteRuleStep, RDD[(Long, Iterable[String])])], + persistFactory: PersistFactory): Unit = { + stepRdds.foreach { stepRdd => + val (step, rdd) = stepRdd + if (step.persistType == RecordPersistType) { + val name = step.name + rdd.foreach { pair => + val (t, items) = pair + val persist = persistFactory.getPersists(t) + persist.persistRecords(items, name) + } + } + } + } + + def updateDataSources(stepRdds: Seq[(ConcreteRuleStep, RDD[(Long, Iterable[String])])], + dataSources: Seq[DataSource]): Unit = { + stepRdds.foreach { stepRdd => + val (step, rdd) = stepRdd + if (step.updateDataSource.nonEmpty) { + val udpateDataSources = dataSources.filter { ds => + step.updateDataSource match { + case Some(dsName) if (dsName == ds.name) => true + case _ => false + } + } + if (udpateDataSources.size > 0) { + val name = step.name + rdd.foreach { pair => + val (t, items) = pair + udpateDataSources.foreach { ds => + ds.dataSourceCacheOpt.foreach(_.updateData(items, t)) + } + } + } + } + } + } } diff --git a/measure/src/main/scala/org/apache/griffin/measure/process/engine/SparkDqEngine.scala b/measure/src/main/scala/org/apache/griffin/measure/process/engine/SparkDqEngine.scala index e00d65951..bdd1b7731 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/process/engine/SparkDqEngine.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/process/engine/SparkDqEngine.scala @@ -23,6 +23,7 @@ import org.apache.griffin.measure.log.Loggable import org.apache.griffin.measure.rule.dsl.{MetricPersistType, RecordPersistType} import org.apache.griffin.measure.rule.step._ import org.apache.griffin.measure.utils.JsonUtil +import org.apache.spark.rdd.RDD import org.apache.spark.sql.{DataFrame, SQLContext} trait SparkDqEngine extends DqEngine { @@ -77,6 +78,35 @@ trait SparkDqEngine extends DqEngine { } } + def collectUpdateRDD(ruleStep: ConcreteRuleStep, timeGroups: Iterable[Long] + ): Option[RDD[(Long, Iterable[String])]] = { + ruleStep match { + case step: ConcreteRuleStep if (step.persistType == RecordPersistType) => { + val name = step.name + try { + val pdf = sqlContext.table(s"`${name}`") + val cols = pdf.columns + val rdd = pdf.flatMap { row => + val values = cols.flatMap { col => + Some((col, row.getAs[Any](col))) + }.toMap + values.get(GroupByColumn.tmst) match { + case Some(t: Long) if (timeGroups.exists(_ == t)) => Some((t, JsonUtil.toJson(values))) + case _ => None + } + }.groupByKey() + Some(rdd) + } catch { + case e: Throwable => { + error(s"collect records ${name} error: ${e.getMessage}") + None + } + } + } + case _ => None + } + } + def collectRecords(ruleStep: ConcreteRuleStep, timeGroups: Iterable[Long]): Map[Long, DataFrame] = { ruleStep match { case step: ConcreteRuleStep if (step.persistType == RecordPersistType) => { diff --git a/measure/src/test/scala/org/apache/griffin/measure/process/BatchProcessTest.scala b/measure/src/test/scala/org/apache/griffin/measure/process/BatchProcessTest.scala index a1e4854b4..5bb092cfc 100644 --- a/measure/src/test/scala/org/apache/griffin/measure/process/BatchProcessTest.scala +++ b/measure/src/test/scala/org/apache/griffin/measure/process/BatchProcessTest.scala @@ -35,8 +35,8 @@ import scala.util.{Failure, Success, Try} class BatchProcessTest extends FunSuite with Matchers with BeforeAndAfter with Loggable { val envFile = "src/test/resources/env-test.json" - val confFile = "src/test/resources/config-test-profiling.json" -// val confFile = "src/test/resources/config-test-accuracy.json" +// val confFile = "src/test/resources/config-test-profiling.json" + val confFile = "src/test/resources/config-test-accuracy.json" val envFsType = "local" val userFsType = "local" From 4fa03b67beac18b3e8d84254f84e06cbde9dd55d Mon Sep 17 00:00:00 2001 From: Lionel Liu Date: Tue, 26 Sep 2017 00:44:03 +0800 Subject: [PATCH 082/111] optimized rdd collection --- .../griffin/measure/persist/HdfsPersist.scala | 57 +++--- .../griffin/measure/persist/HttpPersist.scala | 2 +- .../measure/persist/LoggerPersist.scala | 28 +-- .../measure/persist/MultiPersists.scala | 2 +- .../griffin/measure/persist/Persist.scala | 2 +- .../measure/process/BatchDqProcess.scala | 5 +- .../measure/process/StreamingDqThread.scala | 22 ++- .../measure/process/engine/DqEngine.scala | 6 +- .../measure/process/engine/DqEngines.scala | 163 ++++++------------ .../process/engine/SparkDqEngine.scala | 107 ++++++------ 10 files changed, 178 insertions(+), 216 deletions(-) diff --git a/measure/src/main/scala/org/apache/griffin/measure/persist/HdfsPersist.scala b/measure/src/main/scala/org/apache/griffin/measure/persist/HdfsPersist.scala index d6f927319..431fe1031 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/persist/HdfsPersist.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/persist/HdfsPersist.scala @@ -198,34 +198,34 @@ case class HdfsPersist(config: Map[String, Any], metricName: String, timeStamp: } - def persistRecords(df: DataFrame, name: String): Unit = { - val records = df.toJSON - val path = filePath(name) - try { - val recordCount = records.count - val count = if (maxPersistLines < 0) recordCount else scala.math.min(maxPersistLines, recordCount) - if (count > 0) { - val groupCount = ((count - 1) / maxLinesPerFile + 1).toInt - if (groupCount <= 1) { - val recs = records.take(count.toInt) - persistRecords(path, recs) - } else { - val groupedRecords: RDD[(Long, Iterable[String])] = - records.zipWithIndex.flatMap { r => - val gid = r._2 / maxLinesPerFile - if (gid < groupCount) Some((gid, r._1)) else None - }.groupByKey() - groupedRecords.foreach { group => - val (gid, recs) = group - val hdfsPath = if (gid == 0) path else withSuffix(path, gid.toString) - persistRecords(hdfsPath, recs) - } - } - } - } catch { - case e: Throwable => error(e.getMessage) - } - } +// def persistRecords(df: DataFrame, name: String): Unit = { +// val records = df.toJSON +// val path = filePath(name) +// try { +// val recordCount = records.count +// val count = if (maxPersistLines < 0) recordCount else scala.math.min(maxPersistLines, recordCount) +// if (count > 0) { +// val groupCount = ((count - 1) / maxLinesPerFile + 1).toInt +// if (groupCount <= 1) { +// val recs = records.take(count.toInt) +// persistRecords(path, recs) +// } else { +// val groupedRecords: RDD[(Long, Iterable[String])] = +// records.zipWithIndex.flatMap { r => +// val gid = r._2 / maxLinesPerFile +// if (gid < groupCount) Some((gid, r._1)) else None +// }.groupByKey() +// groupedRecords.foreach { group => +// val (gid, recs) = group +// val hdfsPath = if (gid == 0) path else withSuffix(path, gid.toString) +// persistRecords(hdfsPath, recs) +// } +// } +// } +// } catch { +// case e: Throwable => error(e.getMessage) +// } +// } def persistRecords(records: Iterable[String], name: String): Unit = { val path = filePath(name) @@ -278,6 +278,7 @@ case class HdfsPersist(config: Map[String, Any], metricName: String, timeStamp: def persistMetrics(metrics: Map[String, Any]): Unit = { val json = JsonUtil.toJson(metrics) try { + info(s"${json}") persistRecords(MetricsFile, json :: Nil) } catch { case e: Throwable => error(e.getMessage) diff --git a/measure/src/main/scala/org/apache/griffin/measure/persist/HttpPersist.scala b/measure/src/main/scala/org/apache/griffin/measure/persist/HttpPersist.scala index 3c07a9094..225ee4136 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/persist/HttpPersist.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/persist/HttpPersist.scala @@ -89,7 +89,7 @@ case class HttpPersist(config: Map[String, Any], metricName: String, timeStamp: def log(rt: Long, msg: String): Unit = {} - def persistRecords(df: DataFrame, name: String): Unit = {} +// def persistRecords(df: DataFrame, name: String): Unit = {} def persistRecords(records: Iterable[String], name: String): Unit = {} // def persistMetrics(metrics: Seq[String], name: String): Unit = { diff --git a/measure/src/main/scala/org/apache/griffin/measure/persist/LoggerPersist.scala b/measure/src/main/scala/org/apache/griffin/measure/persist/LoggerPersist.scala index fc565f878..0cd6f6bae 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/persist/LoggerPersist.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/persist/LoggerPersist.scala @@ -117,20 +117,20 @@ case class LoggerPersist(config: Map[String, Any], metricName: String, timeStamp println(s"[${timeStamp}] ${rt}: ${msg}") } - def persistRecords(df: DataFrame, name: String): Unit = { - val records = df.toJSON - println(s"${name} [${timeStamp}] records: ") - try { - val recordCount = records.count.toInt - val count = if (maxLogLines < 0) recordCount else scala.math.min(maxLogLines, recordCount) - if (count > 0) { - val recordsArray = records.take(count) - recordsArray.foreach(println) - } - } catch { - case e: Throwable => error(e.getMessage) - } - } +// def persistRecords(df: DataFrame, name: String): Unit = { +// val records = df.toJSON +// println(s"${name} [${timeStamp}] records: ") +// try { +// val recordCount = records.count.toInt +// val count = if (maxLogLines < 0) recordCount else scala.math.min(maxLogLines, recordCount) +// if (count > 0) { +// val recordsArray = records.take(count) +// recordsArray.foreach(println) +// } +// } catch { +// case e: Throwable => error(e.getMessage) +// } +// } def persistRecords(records: Iterable[String], name: String): Unit = { try { diff --git a/measure/src/main/scala/org/apache/griffin/measure/persist/MultiPersists.scala b/measure/src/main/scala/org/apache/griffin/measure/persist/MultiPersists.scala index 0b8b2f61e..0b7c98c48 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/persist/MultiPersists.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/persist/MultiPersists.scala @@ -50,7 +50,7 @@ case class MultiPersists(persists: Iterable[Persist]) extends Persist { def log(rt: Long, msg: String): Unit = { persists.foreach(_.log(rt, msg)) } - def persistRecords(df: DataFrame, name: String): Unit = { persists.foreach(_.persistRecords(df, name)) } +// def persistRecords(df: DataFrame, name: String): Unit = { persists.foreach(_.persistRecords(df, name)) } def persistRecords(records: Iterable[String], name: String): Unit = { persists.foreach(_.persistRecords(records, name)) } // def persistMetrics(metrics: Seq[String], name: String): Unit = { persists.foreach(_.persistMetrics(metrics, name)) } def persistMetrics(metrics: Map[String, Any]): Unit = { persists.foreach(_.persistMetrics(metrics)) } diff --git a/measure/src/main/scala/org/apache/griffin/measure/persist/Persist.scala b/measure/src/main/scala/org/apache/griffin/measure/persist/Persist.scala index d354a5114..2884fa6d3 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/persist/Persist.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/persist/Persist.scala @@ -43,7 +43,7 @@ trait Persist extends Loggable with Serializable { // def records(recs: RDD[String], tp: String): Unit // def records(recs: Iterable[String], tp: String): Unit - def persistRecords(df: DataFrame, name: String): Unit +// def persistRecords(df: DataFrame, name: String): Unit def persistRecords(records: Iterable[String], name: String): Unit // def persistMetrics(metrics: Seq[String], name: String): Unit def persistMetrics(metrics: Map[String, Any]): Unit diff --git a/measure/src/main/scala/org/apache/griffin/measure/process/BatchDqProcess.scala b/measure/src/main/scala/org/apache/griffin/measure/process/BatchDqProcess.scala index a0a2239ef..2cff9902c 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/process/BatchDqProcess.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/process/BatchDqProcess.scala @@ -94,9 +94,8 @@ case class BatchDqProcess(allParam: AllParam) extends DqProcess { // persist results val timeGroups = dqEngines.persistAllMetrics(ruleSteps, persistFactory) - val stepRdds = dqEngines.collectUpdateRDDs(ruleSteps, timeGroups) - - dqEngines.persistAllRecords(stepRdds, persistFactory) + val rdds = dqEngines.collectUpdateRDDs(ruleSteps, timeGroups) + dqEngines.persistAllRecords(rdds, persistFactory) // dqEngines.persistAllRecords(ruleSteps, persistFactory, timeGroups) // end time diff --git a/measure/src/main/scala/org/apache/griffin/measure/process/StreamingDqThread.scala b/measure/src/main/scala/org/apache/griffin/measure/process/StreamingDqThread.scala index c7224c36d..cf365e834 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/process/StreamingDqThread.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/process/StreamingDqThread.scala @@ -69,19 +69,25 @@ case class StreamingDqThread(dqEngines: DqEngines, // persist results val timeGroups = dqEngines.persistAllMetrics(ruleSteps, persistFactory) - // get rdd of steps - val stepRdds = dqEngines.collectUpdateRDDs(ruleSteps, timeGroups) - - dqEngines.persistAllRecords(stepRdds, persistFactory) - val rt = new Date().getTime - val persistResultTimeStr = s"persist result and records using time: ${rt - ct} ms" + val persistResultTimeStr = s"persist result using time: ${rt - ct} ms" println(persistResultTimeStr) appPersist.log(rt, persistResultTimeStr) + val rdds = dqEngines.collectUpdateRDDs(ruleSteps, timeGroups) + + val lt = new Date().getTime + val collectoRddTimeStr = s"collect records using time: ${lt - rt} ms" + println(collectoRddTimeStr) + appPersist.log(lt, collectoRddTimeStr) + + // persist records + dqEngines.persistAllRecords(rdds, persistFactory) +// dqEngines.persistAllRecords(ruleSteps, persistFactory, timeGroups) + // update data source + dqEngines.updateDataSources(rdds, dataSources) // dqEngines.updateDataSources(ruleSteps, dataSources, timeGroups) - dqEngines.updateDataSources(stepRdds, dataSources) TimeInfoCache.endTimeInfoCache @@ -89,7 +95,7 @@ case class StreamingDqThread(dqEngines: DqEngines, cleanData val et = new Date().getTime - val persistTimeStr = s"update data source using time: ${et - rt} ms" + val persistTimeStr = s"persist records using time: ${et - lt} ms" println(persistTimeStr) appPersist.log(et, persistTimeStr) diff --git a/measure/src/main/scala/org/apache/griffin/measure/process/engine/DqEngine.scala b/measure/src/main/scala/org/apache/griffin/measure/process/engine/DqEngine.scala index c82d318b9..84d591774 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/process/engine/DqEngine.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/process/engine/DqEngine.scala @@ -33,9 +33,9 @@ trait DqEngine extends Loggable with Serializable { def collectMetrics(ruleStep: ConcreteRuleStep): Map[Long, Map[String, Any]] - def collectRecords(ruleStep: ConcreteRuleStep, timeGroups: Iterable[Long]): Map[Long, DataFrame] - - def collectUpdateCacheDatas(ruleStep: ConcreteRuleStep, timeGroups: Iterable[Long]): Map[Long, DataFrame] +// def collectRecords(ruleStep: ConcreteRuleStep, timeGroups: Iterable[Long]): Option[RDD[(Long, Iterable[String])]] +// +// def collectUpdateCacheDatas(ruleStep: ConcreteRuleStep, timeGroups: Iterable[Long]): Option[RDD[(Long, Iterable[String])]] def collectUpdateRDD(ruleStep: ConcreteRuleStep, timeGroups: Iterable[Long]): Option[RDD[(Long, Iterable[String])]] } diff --git a/measure/src/main/scala/org/apache/griffin/measure/process/engine/DqEngines.scala b/measure/src/main/scala/org/apache/griffin/measure/process/engine/DqEngines.scala index d3b4a2b78..1bafa1532 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/process/engine/DqEngines.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/process/engine/DqEngines.scala @@ -33,16 +33,9 @@ case class DqEngines(engines: Seq[DqEngine]) extends DqEngine { val persistOrder: List[PersistType] = List(MetricPersistType, RecordPersistType) def loadData(dataSources: Seq[DataSource], ms: Long): Unit = { -// val dataSources = dataSourceParams.flatMap { param => -// genDataSource(param) -// } dataSources.foreach { ds => ds.loadData(ms) } -// dataSources.flatMap { ds => -// val success = ds.loadData(ms) -// if (success) Some(ds.name) else None -// } } def runRuleSteps(ruleSteps: Seq[ConcreteRuleStep]): Unit = { @@ -75,94 +68,56 @@ case class DqEngines(engines: Seq[DqEngine]) extends DqEngine { updateTimeGroups } - def persistAllRecords(ruleSteps: Seq[ConcreteRuleStep], persistFactory: PersistFactory, - timeGroups: Iterable[Long]): Unit = { - val recordSteps = ruleSteps.filter(_.persistType == RecordPersistType) - recordSteps.foreach { step => - val name = step.name - val records = collectRecords(step, timeGroups) - records.foreach { pair => - val (t, recs) = pair - val persist = persistFactory.getPersists(t) - persist.persistRecords(recs, name) - } - } - } - - def updateDataSources(ruleSteps: Seq[ConcreteRuleStep], dataSources: Seq[DataSource], - timeGroups: Iterable[Long]): Unit = { - val recordSteps = ruleSteps.filter(_.updateDataSource.nonEmpty) - recordSteps.foreach { step => - val udpateDataSources = dataSources.filter { ds => - step.updateDataSource match { - case Some(dsName) if (dsName == ds.name) => true - case _ => false - } - } - if (udpateDataSources.size > 0) { -// val records = collectUpdateCacheDatas(step, timeGroups) -// -// udpateDataSources.foreach { ds => -// ds.updateDataMap(records) -// } - - collectUpdateRDD(step, timeGroups) match { - case Some(rdd) => { - rdd.foreach { pair => - val (t, items) = pair - udpateDataSources.foreach { ds => - ds.dataSourceCacheOpt.foreach(_.updateData(items, t)) - } - } - } - case _ => println("empty update data source") - } - } - } - } - -// def persistAllResults(ruleSteps: Seq[ConcreteRuleStep], persistFactory: PersistFactory): Unit = { -// // 1. persist metric -// val metricSteps = ruleSteps.filter(_.persistType == MetricPersistType) -// val allMetrics: Map[Long, Map[String, Any]] = { -// metricSteps.foldLeft(Map[Long, Map[String, Any]]()) { (ret, step) => -// val metrics = collectMetrics(step) -// metrics.foldLeft(ret) { (total, pair) => -// val (k, v) = pair -// total.get(k) match { -// case Some(map) => total + (k -> (map ++ v)) -// case _ => total + pair +// def persistAllRecords(ruleSteps: Seq[ConcreteRuleStep], persistFactory: PersistFactory, +// timeGroups: Iterable[Long]): Unit = { +// val recordSteps = ruleSteps.filter(_.persistType == RecordPersistType) +// recordSteps.foreach { step => +// collectRecords(step, timeGroups) match { +// case Some(rdd) => { +// val name = step.name +// rdd.foreach { pair => +// val (t, items) = pair +// val persist = persistFactory.getPersists(t) +// persist.persistRecords(items, name) // } // } +// case _ => { +// println(s"empty records to persist") +// } // } // } -// val updateTimeGroups = allMetrics.keys -// allMetrics.foreach { pair => -// val (t, metric) = pair -// val persist = persistFactory.getPersists(t) -// persist.persistMetrics(metric) -// } +// } // -// // 2. persist record -// val recordSteps = ruleSteps.filter(_.persistType == RecordPersistType) -// recordSteps.foreach { step => -// val name = step.name -// val records = collectRecords(step, updateTimeGroups) -// records.foreach { pair => -// val (t, recs) = pair -// val persist = persistFactory.getPersists(t) -// persist.persistRecords(recs, name) +// def updateDataSources(ruleSteps: Seq[ConcreteRuleStep], dataSources: Seq[DataSource], +// timeGroups: Iterable[Long]): Unit = { +// val updateSteps = ruleSteps.filter(_.updateDataSource.nonEmpty) +// updateSteps.foreach { step => +// collectUpdateCacheDatas(step, timeGroups) match { +// case Some(rdd) => { +// val udpateDataSources = dataSources.filter { ds => +// step.updateDataSource match { +// case Some(dsName) if (dsName == ds.name) => true +// case _ => false +// } +// } +// if (udpateDataSources.size > 0) { +// val name = step.name +// rdd.foreach { pair => +// val (t, items) = pair +// udpateDataSources.foreach { ds => +// ds.dataSourceCacheOpt.foreach(_.updateData(items, t)) +// } +// } +// } +// } +// case _ => { +// println(s"empty data source to update") +// } // } // } // } -// def genDataSource(dataSourceParam: DataSourceParam): Option[DirectDataSource] = { -// val ret = engines.foldLeft(None: Option[DirectDataSource]) { (dsOpt, engine) => -// if (dsOpt.isEmpty) engine.genDataSource(dataSourceParam) else dsOpt -// } -// if (ret.isEmpty) warn(s"init data source warn: no dq engine support ${dataSourceParam}") -// ret -// } + /////////////////////////// def runRuleStep(ruleStep: ConcreteRuleStep): Boolean = { val ret = engines.foldLeft(false) { (done, engine) => @@ -172,27 +127,18 @@ case class DqEngines(engines: Seq[DqEngine]) extends DqEngine { ret } -// def persistRecords(ruleStep: ConcreteRuleStep, timeGroups: Iterable[Long], persistFactory: PersistFactory): Boolean = { -// val ret = engines.foldLeft(false) { (done, engine) => -// done || engine.persistRecords(ruleStep, timeGroups, persistFactory) -// } -// if (!ret) error(s"persist result warn: no dq engine support ${ruleStep}") -// ret + /////////////////////////// + +// def collectRecords(ruleStep: ConcreteRuleStep, timeGroups: Iterable[Long]): Option[RDD[(Long, Iterable[String])]] = { +// engines.flatMap { engine => +// engine.collectRecords(ruleStep, timeGroups) +// }.headOption +// } +// def collectUpdateCacheDatas(ruleStep: ConcreteRuleStep, timeGroups: Iterable[Long]): Option[RDD[(Long, Iterable[String])]] = { +// engines.flatMap { engine => +// engine.collectUpdateCacheDatas(ruleStep, timeGroups) +// }.headOption // } - def collectRecords(ruleStep: ConcreteRuleStep, timeGroups: Iterable[Long]): Map[Long, DataFrame] = { - val ret = engines.foldLeft(Map[Long, DataFrame]()) { (ret, engine) => - ret ++ engine.collectRecords(ruleStep, timeGroups) - } -// if (ret.isEmpty) warn(s"collect records warn: no records collected for ${ruleStep}") - ret - } - def collectUpdateCacheDatas(ruleStep: ConcreteRuleStep, timeGroups: Iterable[Long]): Map[Long, DataFrame] = { - val ret = engines.foldLeft(Map[Long, DataFrame]()) { (ret, engine) => - ret ++ engine.collectUpdateCacheDatas(ruleStep, timeGroups) - } - // if (ret.isEmpty) warn(s"collect records warn: no records collected for ${ruleStep}") - ret - } def collectMetrics(ruleStep: ConcreteRuleStep): Map[Long, Map[String, Any]] = { val ret = engines.foldLeft(Map[Long, Map[String, Any]]()) { (ret, engine) => ret ++ engine.collectMetrics(ruleStep) @@ -201,12 +147,15 @@ case class DqEngines(engines: Seq[DqEngine]) extends DqEngine { ret } - def collectUpdateRDD(ruleStep: ConcreteRuleStep, timeGroups: Iterable[Long]): Option[RDD[(Long, Iterable[String])]] = { + def collectUpdateRDD(ruleStep: ConcreteRuleStep, timeGroups: Iterable[Long] + ): Option[RDD[(Long, Iterable[String])]] = { engines.flatMap { engine => engine.collectUpdateRDD(ruleStep, timeGroups) }.headOption } + //////////////////////////// + def collectUpdateRDDs(ruleSteps: Seq[ConcreteRuleStep], timeGroups: Iterable[Long] ): Seq[(ConcreteRuleStep, RDD[(Long, Iterable[String])])] = { ruleSteps.flatMap { rs => diff --git a/measure/src/main/scala/org/apache/griffin/measure/process/engine/SparkDqEngine.scala b/measure/src/main/scala/org/apache/griffin/measure/process/engine/SparkDqEngine.scala index bdd1b7731..ee994fdc2 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/process/engine/SparkDqEngine.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/process/engine/SparkDqEngine.scala @@ -81,7 +81,8 @@ trait SparkDqEngine extends DqEngine { def collectUpdateRDD(ruleStep: ConcreteRuleStep, timeGroups: Iterable[Long] ): Option[RDD[(Long, Iterable[String])]] = { ruleStep match { - case step: ConcreteRuleStep if (step.persistType == RecordPersistType) => { + case step: ConcreteRuleStep if ((step.persistType == RecordPersistType) + || (step.updateDataSource.nonEmpty)) => { val name = step.name try { val pdf = sqlContext.table(s"`${name}`") @@ -107,54 +108,60 @@ trait SparkDqEngine extends DqEngine { } } - def collectRecords(ruleStep: ConcreteRuleStep, timeGroups: Iterable[Long]): Map[Long, DataFrame] = { - ruleStep match { - case step: ConcreteRuleStep if (step.persistType == RecordPersistType) => { - val name = step.name - try { - val pdf = sqlContext.table(s"`${name}`") - timeGroups.flatMap { timeGroup => - try { - val tdf = pdf.filter(s"`${GroupByColumn.tmst}` = ${timeGroup}") - Some((timeGroup, tdf)) - } catch { - case e: Throwable => None - } - }.toMap - } catch { - case e: Throwable => { - error(s"collect records ${name} error: ${e.getMessage}") - Map[Long, DataFrame]() - } - } - } - case _ => Map[Long, DataFrame]() - } - } - - def collectUpdateCacheDatas(ruleStep: ConcreteRuleStep, timeGroups: Iterable[Long]): Map[Long, DataFrame] = { - ruleStep match { - case step: ConcreteRuleStep if (step.updateDataSource.nonEmpty) => { - val name = step.name - try { - val pdf = sqlContext.table(s"`${name}`") - timeGroups.flatMap { timeGroup => - try { - val tdf = pdf.filter(s"`${GroupByColumn.tmst}` = ${timeGroup}") - Some((timeGroup, tdf)) - } catch { - case e: Throwable => None - } - }.toMap - } catch { - case e: Throwable => { - error(s"collect update cache datas ${name} error: ${e.getMessage}") - Map[Long, DataFrame]() - } - } - } - case _ => Map[Long, DataFrame]() - } - } +// def collectRecords(ruleStep: ConcreteRuleStep, timeGroups: Iterable[Long]): Option[RDD[(Long, Iterable[String])]] = { +// ruleStep match { +// case step: ConcreteRuleStep if (step.persistType == RecordPersistType) => { +// val name = step.name +// try { +// val pdf = sqlContext.table(s"`${name}`") +// val cols = pdf.columns +// val rdd = pdf.flatMap { row => +// val values = cols.flatMap { col => +// Some((col, row.getAs[Any](col))) +// }.toMap +// values.get(GroupByColumn.tmst) match { +// case Some(t: Long) if (timeGroups.exists(_ == t)) => Some((t, JsonUtil.toJson(values))) +// case _ => None +// } +// }.groupByKey() +// Some(rdd) +// } catch { +// case e: Throwable => { +// error(s"collect records ${name} error: ${e.getMessage}") +// None +// } +// } +// } +// case _ => None +// } +// } +// +// def collectUpdateCacheDatas(ruleStep: ConcreteRuleStep, timeGroups: Iterable[Long]): Option[RDD[(Long, Iterable[String])]] = { +// ruleStep match { +// case step: ConcreteRuleStep if (step.updateDataSource.nonEmpty) => { +// val name = step.name +// try { +// val pdf = sqlContext.table(s"`${name}`") +// val cols = pdf.columns +// val rdd = pdf.flatMap { row => +// val values = cols.flatMap { col => +// Some((col, row.getAs[Any](col))) +// }.toMap +// values.get(GroupByColumn.tmst) match { +// case Some(t: Long) if (timeGroups.exists(_ == t)) => Some((t, JsonUtil.toJson(values))) +// case _ => None +// } +// }.groupByKey() +// Some(rdd) +// } catch { +// case e: Throwable => { +// error(s"collect update cache datas ${name} error: ${e.getMessage}") +// None +// } +// } +// } +// case _ => None +// } +// } } From 2a7ea805441a793cf64781469f485d28fdd6622a Mon Sep 17 00:00:00 2001 From: Lionel Liu Date: Tue, 26 Sep 2017 09:24:06 +0800 Subject: [PATCH 083/111] cache --- .../apache/griffin/measure/process/BatchDqProcess.scala | 4 ++++ .../griffin/measure/process/StreamingDqThread.scala | 8 ++++++++ 2 files changed, 12 insertions(+) diff --git a/measure/src/main/scala/org/apache/griffin/measure/process/BatchDqProcess.scala b/measure/src/main/scala/org/apache/griffin/measure/process/BatchDqProcess.scala index 2cff9902c..737a43f10 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/process/BatchDqProcess.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/process/BatchDqProcess.scala @@ -95,9 +95,13 @@ case class BatchDqProcess(allParam: AllParam) extends DqProcess { val timeGroups = dqEngines.persistAllMetrics(ruleSteps, persistFactory) val rdds = dqEngines.collectUpdateRDDs(ruleSteps, timeGroups) + rdds.foreach(_._2.cache()) + dqEngines.persistAllRecords(rdds, persistFactory) // dqEngines.persistAllRecords(ruleSteps, persistFactory, timeGroups) + rdds.foreach(_._2.unpersist()) + // end time val endTime = new Date().getTime persist.log(endTime, s"process using time: ${endTime - startTime} ms") diff --git a/measure/src/main/scala/org/apache/griffin/measure/process/StreamingDqThread.scala b/measure/src/main/scala/org/apache/griffin/measure/process/StreamingDqThread.scala index cf365e834..df1cc1b53 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/process/StreamingDqThread.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/process/StreamingDqThread.scala @@ -75,6 +75,12 @@ case class StreamingDqThread(dqEngines: DqEngines, appPersist.log(rt, persistResultTimeStr) val rdds = dqEngines.collectUpdateRDDs(ruleSteps, timeGroups) + rdds.foreach(_._2.cache()) + rdds.foreach { pr => + val (step, rdd) = pr + val cnt = rdd.count + println(s"step [${step.name}] group count: ${cnt}") + } val lt = new Date().getTime val collectoRddTimeStr = s"collect records using time: ${lt - rt} ms" @@ -89,6 +95,8 @@ case class StreamingDqThread(dqEngines: DqEngines, dqEngines.updateDataSources(rdds, dataSources) // dqEngines.updateDataSources(ruleSteps, dataSources, timeGroups) + rdds.foreach(_._2.unpersist()) + TimeInfoCache.endTimeInfoCache // clean old data From 242a31ec049ae62eaf7f545840c7a18054b79263 Mon Sep 17 00:00:00 2001 From: Lionel Liu Date: Thu, 28 Sep 2017 18:14:39 +0800 Subject: [PATCH 084/111] streaming profiling min partition unit modify --- .../measure/data/source/DataSourceCache.scala | 2 +- .../rule/adaptor/GriffinDslAdaptor.scala | 9 ++- .../config-test-profiling-streaming.json | 2 +- .../rule/adaptor/GriffinDslAdaptorTest.scala | 65 +++++++++++++++++++ 4 files changed, 75 insertions(+), 3 deletions(-) create mode 100644 measure/src/test/scala/org/apache/griffin/measure/rule/adaptor/GriffinDslAdaptorTest.scala diff --git a/measure/src/main/scala/org/apache/griffin/measure/data/source/DataSourceCache.scala b/measure/src/main/scala/org/apache/griffin/measure/data/source/DataSourceCache.scala index fc798e6cf..769550ff1 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/data/source/DataSourceCache.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/data/source/DataSourceCache.scala @@ -64,7 +64,7 @@ case class DataSourceCache(sqlContext: SQLContext, param: Map[String, Any], } val rowSepLiteral = "\n" - val partitionUnits: List[String] = List("hour", "min") + val partitionUnits: List[String] = List("hour", "min", "sec") val newCacheLock = InfoCacheInstance.genLock(s"${cacheInfoPath}.new") val oldCacheLock = InfoCacheInstance.genLock(s"${cacheInfoPath}.old") diff --git a/measure/src/main/scala/org/apache/griffin/measure/rule/adaptor/GriffinDslAdaptor.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/adaptor/GriffinDslAdaptor.scala index 36cb25ab3..5e3b6321d 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rule/adaptor/GriffinDslAdaptor.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rule/adaptor/GriffinDslAdaptor.scala @@ -315,7 +315,14 @@ case class GriffinDslAdaptor(dataSourceNames: Seq[String], } private def checkDataSourceExists(name: String): Boolean = { - RuleAdaptorGroup.dataChecker.existDataSourceName(name) + try { + RuleAdaptorGroup.dataChecker.existDataSourceName(name) + } catch { + case e: Throwable => { + error(s"check data source exists error: ${e.getMessage}") + false + } + } } } diff --git a/measure/src/test/resources/config-test-profiling-streaming.json b/measure/src/test/resources/config-test-profiling-streaming.json index 5a35230a0..1605ed5b0 100644 --- a/measure/src/test/resources/config-test-profiling-streaming.json +++ b/measure/src/test/resources/config-test-profiling-streaming.json @@ -44,7 +44,7 @@ "info.path": "source", "ready.time.interval": "10s", "ready.time.delay": "0", - "time.range": ["-1m", "0"] + "time.range": ["0", "0"] } } ], diff --git a/measure/src/test/scala/org/apache/griffin/measure/rule/adaptor/GriffinDslAdaptorTest.scala b/measure/src/test/scala/org/apache/griffin/measure/rule/adaptor/GriffinDslAdaptorTest.scala new file mode 100644 index 000000000..f27376a64 --- /dev/null +++ b/measure/src/test/scala/org/apache/griffin/measure/rule/adaptor/GriffinDslAdaptorTest.scala @@ -0,0 +1,65 @@ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, +software distributed under the License is distributed on an +"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +KIND, either express or implied. See the License for the +specific language governing permissions and limitations +under the License. +*/ +package org.apache.griffin.measure.rule.adaptor + +import org.apache.griffin.measure.process.check.DataChecker +import org.apache.griffin.measure.utils.JsonUtil +import org.junit.runner.RunWith +import org.scalatest.junit.JUnitRunner +import org.scalatest.{BeforeAndAfter, FunSuite, Matchers} +import org.scalamock.scalatest.MockFactory + +@RunWith(classOf[JUnitRunner]) +class GriffinDslAdaptorTest extends FunSuite with Matchers with BeforeAndAfter with MockFactory { + + test ("profiling groupby") { + val adaptor = GriffinDslAdaptor("source" :: Nil, "count" :: Nil, RunPhase) + + val ruleJson = + """ + |{ + | "dsl.type": "griffin-dsl", + | "dq.type": "profiling", + | "rule": "source.user_id.count() as cnt", + | "details": { + | "source": "source", + | "profiling": { + | "name": "count", + | "persist.type": "metric" + | } + | } + |} + """.stripMargin + + // rule: Map[String, Any] + val rule: Map[String, Any] = JsonUtil.toAnyMap(ruleJson) + println(rule) + + val dataCheckerMock = mock[DataChecker] + dataCheckerMock.existDataSourceName _ expects ("source") returning (true) + RuleAdaptorGroup.dataChecker = dataCheckerMock + + val steps = adaptor.genConcreteRuleStep(rule) + + steps.foreach { step => + println(s"${step.name} [${step.dslType}]: ${step.rule}") + } + } + +} From 73e18076f9c02069646eedc799e3c13405038670 Mon Sep 17 00:00:00 2001 From: Lionel Liu Date: Fri, 29 Sep 2017 10:57:29 +0800 Subject: [PATCH 085/111] fix groupby bug --- .../rule/adaptor/GriffinDslAdaptor.scala | 15 +++++-- .../rule/dsl/analyzer/ProfilingAnalyzer.scala | 11 ++--- .../rule/dsl/expr/ClauseExpression.scala | 45 +++++++++++++++++-- .../rule/dsl/parser/GriffinDslParser.scala | 18 +++++++- .../griffin/measure/utils/HdfsUtil.scala | 2 +- .../config-test-profiling-streaming.json | 4 +- .../process/StreamingProcessTest.scala | 4 +- .../rule/adaptor/GriffinDslAdaptorTest.scala | 2 +- 8 files changed, 81 insertions(+), 20 deletions(-) diff --git a/measure/src/main/scala/org/apache/griffin/measure/rule/adaptor/GriffinDslAdaptor.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/adaptor/GriffinDslAdaptor.scala index 5e3b6321d..a17e46603 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rule/adaptor/GriffinDslAdaptor.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rule/adaptor/GriffinDslAdaptor.scala @@ -264,7 +264,7 @@ case class GriffinDslAdaptor(dataSourceNames: Seq[String], case Some(name) => name case _ => dataSourceNames.head } - val analyzer = ProfilingAnalyzer(expr.asInstanceOf[CombinedClause], sourceName) + val analyzer = ProfilingAnalyzer(expr.asInstanceOf[ProfilingClause], sourceName) val selClause = analyzer.selectionExprs.map { sel => val alias = sel match { @@ -274,14 +274,23 @@ case class GriffinDslAdaptor(dataSourceNames: Seq[String], s"${sel.desc}${alias}" }.mkString(", ") - val tailClause = analyzer.tailsExprs.map(_.desc).mkString(" ") +// val tailClause = analyzer.tailsExprs.map(_.desc).mkString(" ") + val tmstGroupbyClause = GroupbyClause(LiteralStringExpr(s"`${GroupByColumn.tmst}`") :: Nil, None) + val mergedGroubbyClause = tmstGroupbyClause.merge(analyzer.groupbyExprOpt match { + case Some(gbc) => gbc + case _ => GroupbyClause(Nil, None) + }) + val groupbyClause = mergedGroubbyClause.desc + val preGroupbyClause = analyzer.preGroupbyExprs.map(_.desc).mkString(" ") + val postGroupbyClause = analyzer.postGroupbyExprs.map(_.desc).mkString(" ") if (!checkDataSourceExists(sourceName)) { Nil } else { // 1. select statement val profilingSql = { - s"SELECT `${GroupByColumn.tmst}`, ${selClause} FROM ${sourceName} ${tailClause} GROUP BY `${GroupByColumn.tmst}`" +// s"SELECT `${GroupByColumn.tmst}`, ${selClause} FROM ${sourceName} ${tailClause} GROUP BY `${GroupByColumn.tmst}`" + s"SELECT `${GroupByColumn.tmst}`, ${selClause} FROM ${sourceName} ${preGroupbyClause} ${groupbyClause} ${postGroupbyClause}" } val profilingMetricName = resultName(details, ProfilingInfo._Profiling) val profilingStep = SparkSqlStep( diff --git a/measure/src/main/scala/org/apache/griffin/measure/rule/dsl/analyzer/ProfilingAnalyzer.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/dsl/analyzer/ProfilingAnalyzer.scala index 47b77037a..df1beeed4 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rule/dsl/analyzer/ProfilingAnalyzer.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rule/dsl/analyzer/ProfilingAnalyzer.scala @@ -21,7 +21,7 @@ package org.apache.griffin.measure.rule.dsl.analyzer import org.apache.griffin.measure.rule.dsl.expr._ -case class ProfilingAnalyzer(expr: CombinedClause, sourceName: String) extends BasicAnalyzer { +case class ProfilingAnalyzer(expr: ProfilingClause, sourceName: String) extends BasicAnalyzer { val dataSourceNames = expr.preOrderTraverseDepthFirst(Set[String]())(seqDataSourceNames, combDataSourceNames) @@ -30,9 +30,10 @@ case class ProfilingAnalyzer(expr: CombinedClause, sourceName: String) extends B expr.selectClause.preOrderTraverseDepthFirst(Seq[SelectionExpr]())(seq, combSelectionExprs) } - val selectionExprs = { - expr.selectClause.exprs.map(_.extractSelf) - } - val tailsExprs = expr.tails.map(_.extractSelf) + val selectionExprs = expr.selectClause.exprs.map(_.extractSelf) + + val groupbyExprOpt = expr.groupbyClauseOpt + val preGroupbyExprs = expr.preGroupbyClauses.map(_.extractSelf) + val postGroupbyExprs = expr.postGroupbyClauses.map(_.extractSelf) } diff --git a/measure/src/main/scala/org/apache/griffin/measure/rule/dsl/expr/ClauseExpression.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/dsl/expr/ClauseExpression.scala index e7bd84f1d..26882b493 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rule/dsl/expr/ClauseExpression.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rule/dsl/expr/ClauseExpression.scala @@ -46,20 +46,32 @@ case class GroupbyClause(exprs: Seq[Expr], havingClauseOpt: Option[Expr]) extend def desc: String = { val gbs = exprs.map(_.desc).mkString(", ") havingClauseOpt match { -// case Some(having) => s"GROUP BY ${gbs} HAVING ${having.desc}" - case Some(having) => s"GROUP BY ${gbs}" + case Some(having) => s"GROUP BY ${gbs} HAVING ${having.desc}" case _ => s"GROUP BY ${gbs}" } } def coalesceDesc: String = { val gbs = exprs.map(_.desc).mkString(", ") havingClauseOpt match { -// case Some(having) => s"GROUP BY ${gbs} HAVING ${having.coalesceDesc}" - case Some(having) => s"GROUP BY ${gbs}" + case Some(having) => s"GROUP BY ${gbs} HAVING ${having.coalesceDesc}" case _ => s"GROUP BY ${gbs}" } } + def merge(other: GroupbyClause): GroupbyClause = { + val newHavingClauseOpt = (havingClauseOpt, other.havingClauseOpt) match { + case (Some(hc), Some(ohc)) => { + val logical1 = LogicalFactorExpr(hc, false, None) + val logical2 = LogicalFactorExpr(ohc, false, None) + Some(BinaryLogicalExpr(logical1, ("AND", logical2) :: Nil)) + } + case (a @ Some(_), _) => a + case (_, b @ Some(_)) => b + case (_, _) => None + } + GroupbyClause(exprs ++ other.exprs, newHavingClauseOpt) + } + } case class OrderbyItem(expr: Expr, orderOpt: Option[String]) extends Expr { @@ -110,4 +122,29 @@ case class CombinedClause(selectClause: SelectClause, tails: Seq[ClauseExpressio s"${head} ${tail.coalesceDesc}" } } +} + +case class ProfilingClause(selectClause: SelectClause, groupbyClauseOpt: Option[GroupbyClause], + preGroupbyClauses: Seq[ClauseExpression], + postGroupbyClauses: Seq[ClauseExpression] + ) extends ClauseExpression { + addChildren(groupbyClauseOpt match { + case Some(gc) => (selectClause +: preGroupbyClauses) ++ (gc +: postGroupbyClauses) + case _ => (selectClause +: preGroupbyClauses) ++ postGroupbyClauses + }) + + def desc: String = { + val selectDesc = selectClause.desc + val groupbyDesc = groupbyClauseOpt.map(_.desc).mkString(" ") + val preDesc = preGroupbyClauses.map(_.desc).mkString(" ") + val postDesc = postGroupbyClauses.map(_.desc).mkString(" ") + s"${selectDesc} ${preDesc} ${groupbyDesc} ${postDesc}" + } + def coalesceDesc: String = { + val selectDesc = selectClause.coalesceDesc + val groupbyDesc = groupbyClauseOpt.map(_.coalesceDesc).mkString(" ") + val preDesc = preGroupbyClauses.map(_.coalesceDesc).mkString(" ") + val postDesc = postGroupbyClauses.map(_.coalesceDesc).mkString(" ") + s"${selectDesc} ${preDesc} ${groupbyDesc} ${postDesc}" + } } \ No newline at end of file diff --git a/measure/src/main/scala/org/apache/griffin/measure/rule/dsl/parser/GriffinDslParser.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/dsl/parser/GriffinDslParser.scala index caa995659..637decbff 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rule/dsl/parser/GriffinDslParser.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rule/dsl/parser/GriffinDslParser.scala @@ -19,15 +19,29 @@ under the License. package org.apache.griffin.measure.rule.dsl.parser import org.apache.griffin.measure.rule.dsl._ -import org.apache.griffin.measure.rule.dsl.expr.Expr +import org.apache.griffin.measure.rule.dsl.expr._ case class GriffinDslParser(dataSourceNames: Seq[String], functionNames: Seq[String] ) extends BasicParser { + /** + * -- profiling clauses -- + * = [ ]+ [ ]+ [ ]+ [ ]+ + */ + + def profilingClause: Parser[ProfilingClause] = selectClause ~ opt(whereClause) ~ + opt(groupbyClause) ~ opt(orderbyClause) ~ opt(limitClause) ^^ { + case sel ~ whereOpt ~ groupbyOpt ~ orderbyOpt ~ limitOpt => { + val preClauses = Seq(whereOpt).flatMap(opt => opt) + val postClauses = Seq(orderbyOpt, limitOpt).flatMap(opt => opt) + ProfilingClause(sel, groupbyOpt, preClauses, postClauses) + } + } + def parseRule(rule: String, dqType: DqType): ParseResult[Expr] = { val rootExpr = dqType match { case AccuracyType => logicalExpression - case ProfilingType => combinedClause + case ProfilingType => profilingClause case _ => expression } parseAll(rootExpr, rule) diff --git a/measure/src/main/scala/org/apache/griffin/measure/utils/HdfsUtil.scala b/measure/src/main/scala/org/apache/griffin/measure/utils/HdfsUtil.scala index 9fa6bcfbe..69f63beed 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/utils/HdfsUtil.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/utils/HdfsUtil.scala @@ -28,7 +28,7 @@ object HdfsUtil extends Loggable { private val conf = new Configuration() conf.set("dfs.support.append", "true") -// conf.set("fs.defaultFS", "hdfs://localhost") // debug @localhost + conf.set("fs.defaultFS", "hdfs://localhost") // debug @localhost private val dfs = FileSystem.get(conf) diff --git a/measure/src/test/resources/config-test-profiling-streaming.json b/measure/src/test/resources/config-test-profiling-streaming.json index 1605ed5b0..4ad165e6d 100644 --- a/measure/src/test/resources/config-test-profiling-streaming.json +++ b/measure/src/test/resources/config-test-profiling-streaming.json @@ -54,11 +54,11 @@ { "dsl.type": "griffin-dsl", "dq.type": "profiling", - "rule": "source.name.count(), source.age.avg(), source.age.max(), source.age.min()", + "rule": "source.name.count(), source.age.avg(), source.age.max(), source.age.min() group by source.name having source.name.count() > 0 limit 2", "details": { "source": "source", "profiling": { - "name": "count", + "name": "prof", "persist.type": "metric" } } diff --git a/measure/src/test/scala/org/apache/griffin/measure/process/StreamingProcessTest.scala b/measure/src/test/scala/org/apache/griffin/measure/process/StreamingProcessTest.scala index 07b7c5e08..73ae06a44 100644 --- a/measure/src/test/scala/org/apache/griffin/measure/process/StreamingProcessTest.scala +++ b/measure/src/test/scala/org/apache/griffin/measure/process/StreamingProcessTest.scala @@ -36,8 +36,8 @@ class StreamingProcessTest extends FunSuite with Matchers with BeforeAndAfter wi val envFile = "src/test/resources/env-streaming.json" // val confFile = "src/test/resources/config-test-accuracy-streaming-multids.json" - val confFile = "src/test/resources/config-test-accuracy-streaming.json" -// val confFile = "src/test/resources/config-test-profiling-streaming.json" +// val confFile = "src/test/resources/config-test-accuracy-streaming.json" + val confFile = "src/test/resources/config-test-profiling-streaming.json" val envFsType = "local" val userFsType = "local" diff --git a/measure/src/test/scala/org/apache/griffin/measure/rule/adaptor/GriffinDslAdaptorTest.scala b/measure/src/test/scala/org/apache/griffin/measure/rule/adaptor/GriffinDslAdaptorTest.scala index f27376a64..a2738df55 100644 --- a/measure/src/test/scala/org/apache/griffin/measure/rule/adaptor/GriffinDslAdaptorTest.scala +++ b/measure/src/test/scala/org/apache/griffin/measure/rule/adaptor/GriffinDslAdaptorTest.scala @@ -36,7 +36,7 @@ class GriffinDslAdaptorTest extends FunSuite with Matchers with BeforeAndAfter w |{ | "dsl.type": "griffin-dsl", | "dq.type": "profiling", - | "rule": "source.user_id.count() as cnt", + | "rule": "source.user_id.count() as cnt group by source.age having source.desc.count() > 5", | "details": { | "source": "source", | "profiling": { From 96519ecf17383fe2e1d70fc993ebed0bf7bfd448 Mon Sep 17 00:00:00 2001 From: Lionel Liu Date: Fri, 29 Sep 2017 13:14:20 +0800 Subject: [PATCH 086/111] a --- .../measure/rule/adaptor/GriffinDslAdaptor.scala | 11 ++++++----- .../measure/rule/adaptor/GriffinDslAdaptorTest.scala | 4 ++-- 2 files changed, 8 insertions(+), 7 deletions(-) diff --git a/measure/src/main/scala/org/apache/griffin/measure/rule/adaptor/GriffinDslAdaptor.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/adaptor/GriffinDslAdaptor.scala index a17e46603..3cd6c2bf4 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rule/adaptor/GriffinDslAdaptor.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rule/adaptor/GriffinDslAdaptor.scala @@ -266,13 +266,14 @@ case class GriffinDslAdaptor(dataSourceNames: Seq[String], } val analyzer = ProfilingAnalyzer(expr.asInstanceOf[ProfilingClause], sourceName) - val selClause = analyzer.selectionExprs.map { sel => + val selExprDescs = analyzer.selectionExprs.map { sel => val alias = sel match { - case s: AliasableExpr if (s.alias.nonEmpty) => s" AS ${s.alias.get}" + case s: AliasableExpr if (s.alias.nonEmpty) => s" AS `${s.alias.get}`" case _ => "" } s"${sel.desc}${alias}" - }.mkString(", ") + } + val selClause = (s"`${GroupByColumn.tmst}`" +: selExprDescs).mkString(", ") // val tailClause = analyzer.tailsExprs.map(_.desc).mkString(" ") val tmstGroupbyClause = GroupbyClause(LiteralStringExpr(s"`${GroupByColumn.tmst}`") :: Nil, None) @@ -290,7 +291,7 @@ case class GriffinDslAdaptor(dataSourceNames: Seq[String], // 1. select statement val profilingSql = { // s"SELECT `${GroupByColumn.tmst}`, ${selClause} FROM ${sourceName} ${tailClause} GROUP BY `${GroupByColumn.tmst}`" - s"SELECT `${GroupByColumn.tmst}`, ${selClause} FROM ${sourceName} ${preGroupbyClause} ${groupbyClause} ${postGroupbyClause}" + s"SELECT ${selClause} FROM ${sourceName} ${preGroupbyClause} ${groupbyClause} ${postGroupbyClause}" } val profilingMetricName = resultName(details, ProfilingInfo._Profiling) val profilingStep = SparkSqlStep( @@ -303,7 +304,7 @@ case class GriffinDslAdaptor(dataSourceNames: Seq[String], // 2. clear processed data val clearDataSourceStep = DfOprStep( - s"${profilingMetricName}_clear", + s"${sourceName}_clear", "clear", Map[String, Any]( ("df.name" -> sourceName) diff --git a/measure/src/test/scala/org/apache/griffin/measure/rule/adaptor/GriffinDslAdaptorTest.scala b/measure/src/test/scala/org/apache/griffin/measure/rule/adaptor/GriffinDslAdaptorTest.scala index a2738df55..85e420f8e 100644 --- a/measure/src/test/scala/org/apache/griffin/measure/rule/adaptor/GriffinDslAdaptorTest.scala +++ b/measure/src/test/scala/org/apache/griffin/measure/rule/adaptor/GriffinDslAdaptorTest.scala @@ -36,11 +36,11 @@ class GriffinDslAdaptorTest extends FunSuite with Matchers with BeforeAndAfter w |{ | "dsl.type": "griffin-dsl", | "dq.type": "profiling", - | "rule": "source.user_id.count() as cnt group by source.age having source.desc.count() > 5", + | "rule": "source.user_id.count() as cnt group by source.age having source.desc.count() > 5 order by source.user_id", | "details": { | "source": "source", | "profiling": { - | "name": "count", + | "name": "prof", | "persist.type": "metric" | } | } From 76837a33c084ccbf76202315987f42cad00ef7d2 Mon Sep 17 00:00:00 2001 From: Lionel Liu Date: Fri, 29 Sep 2017 14:24:08 +0800 Subject: [PATCH 087/111] fix order by bug --- .../rule/adaptor/GriffinDslAdaptor.scala | 1 + .../measure/rule/dsl/expr/SelectExpr.scala | 5 ++ .../measure/rule/dsl/parser/BasicParser.scala | 48 +++++++++++-------- .../config-test-profiling-streaming.json | 2 +- .../test/resources/config-test-profiling.json | 2 +- .../measure/process/BatchProcessTest.scala | 4 +- .../rule/adaptor/GriffinDslAdaptorTest.scala | 2 +- .../rule/dsl/parser/BasicParserTest.scala | 7 +++ 8 files changed, 46 insertions(+), 25 deletions(-) diff --git a/measure/src/main/scala/org/apache/griffin/measure/rule/adaptor/GriffinDslAdaptor.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/adaptor/GriffinDslAdaptor.scala index 3cd6c2bf4..2efde0651 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rule/adaptor/GriffinDslAdaptor.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rule/adaptor/GriffinDslAdaptor.scala @@ -120,6 +120,7 @@ case class GriffinDslAdaptor(dataSourceNames: Seq[String], val result = parser.parseRule(rule, dqType) if (result.successful) Some(result.get) else { + println(result) warn(s"adapt concrete rule step warn: parse rule [ ${rule} ] fails") None } diff --git a/measure/src/main/scala/org/apache/griffin/measure/rule/dsl/expr/SelectExpr.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/dsl/expr/SelectExpr.scala index 039d21591..c2c605dbb 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rule/dsl/expr/SelectExpr.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rule/dsl/expr/SelectExpr.scala @@ -27,6 +27,11 @@ case class DataSourceHeadExpr(name: String) extends HeadExpr { def coalesceDesc: String = desc } +case class FieldNameHeadExpr(field: String) extends HeadExpr { + def desc: String = field + def coalesceDesc: String = desc +} + case class OtherHeadExpr(expr: Expr) extends HeadExpr { addChild(expr) diff --git a/measure/src/main/scala/org/apache/griffin/measure/rule/dsl/parser/BasicParser.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/dsl/parser/BasicParser.scala index 7d13e03b4..12d1f0cad 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rule/dsl/parser/BasicParser.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rule/dsl/parser/BasicParser.scala @@ -94,14 +94,14 @@ trait BasicParser extends JavaTokenParsers with Serializable { val MATH_UNARY: Parser[String] = "+" | "-" val MATH_BINARIES: Seq[Parser[String]] = Seq(("*" | "/" | "%"), ("+" | "-")) - val NOT: Parser[String] = """(?i)not""".r | "!" - val AND: Parser[String] = """(?i)and""".r | "&&" - val OR: Parser[String] = """(?i)or""".r | "||" - val IN: Parser[String] = """(?i)in""".r - val BETWEEN: Parser[String] = """(?i)between""".r - val AND_ONLY: Parser[String] = """(?i)and""".r - val IS: Parser[String] = """(?i)is""".r - val LIKE: Parser[String] = """(?i)like""".r + val NOT: Parser[String] = """(?i)not\s""".r | "!" + val AND: Parser[String] = """(?i)and\s""".r | "&&" + val OR: Parser[String] = """(?i)or\s""".r | "||" + val IN: Parser[String] = """(?i)in\s""".r + val BETWEEN: Parser[String] = """(?i)between\s""".r + val AND_ONLY: Parser[String] = """(?i)and\s""".r + val IS: Parser[String] = """(?i)is\s""".r + val LIKE: Parser[String] = """(?i)like\s""".r val COMPARE: Parser[String] = "=" | "!=" | "<>" | "<=" | ">=" | "<" | ">" val LOGICAL_UNARY: Parser[String] = NOT val LOGICAL_BINARIES: Seq[Parser[String]] = Seq((COMPARE), (AND), (OR)) @@ -118,15 +118,15 @@ trait BasicParser extends JavaTokenParsers with Serializable { val UQUOTE: Parser[String] = "`" val COMMA: Parser[String] = "," - val AS: Parser[String] = "(?i)as".r - val WHERE: Parser[String] = "(?i)where".r - val GROUP: Parser[String] = "(?i)group".r - val ORDER: Parser[String] = "(?i)order".r - val BY: Parser[String] = "(?i)by".r - val DESC: Parser[String] = "(?i)desc".r - val ASC: Parser[String] = "(?i)asc".r - val HAVING: Parser[String] = "(?i)having".r - val LIMIT: Parser[String] = "(?i)limit".r + val AS: Parser[String] = """(?i)as\s""".r + val WHERE: Parser[String] = """(?i)where\s""".r + val GROUP: Parser[String] = """(?i)group\s""".r + val ORDER: Parser[String] = """(?i)order\s""".r + val BY: Parser[String] = """(?i)by\s""".r + val DESC: Parser[String] = """(?i)desc""".r + val ASC: Parser[String] = """(?i)asc""".r + val HAVING: Parser[String] = """(?i)having\s""".r + val LIMIT: Parser[String] = """(?i)limit\s""".r } import Operator._ @@ -177,7 +177,13 @@ trait BasicParser extends JavaTokenParsers with Serializable { def selection: Parser[SelectionExpr] = selectionHead ~ rep(selector) ~ opt(asAlias) ^^ { case head ~ sels ~ aliasOpt => SelectionExpr(head, sels, aliasOpt) } - def selectionHead: Parser[HeadExpr] = DataSourceName ^^ { DataSourceHeadExpr(_) } | function ^^ { OtherHeadExpr(_) } + def selectionHead: Parser[HeadExpr] = DataSourceName ^^ { + DataSourceHeadExpr(_) + } | function ^^ { + OtherHeadExpr(_) + } | TableFieldName ^^ { + FieldNameHeadExpr(_) + } def selector: Parser[SelectExpr] = functionSelect | fieldSelect | indexSelect def fieldSelect: Parser[FieldSelectExpr] = DOT ~> TableFieldName ^^ { FieldSelectExpr(_) } def indexSelect: Parser[IndexSelectExpr] = LSQBR ~> argument <~ RSQBR ^^ { IndexSelectExpr(_) } @@ -307,7 +313,9 @@ trait BasicParser extends JavaTokenParsers with Serializable { def orderbyItem: Parser[OrderbyItem] = expression ~ opt(DESC | ASC) ^^ { case expr ~ orderOpt => OrderbyItem(expr, orderOpt) } - def orderbyClause: Parser[OrderbyClause] = ORDER ~> BY ~> rep1sep(orderbyItem, COMMA) ^^ { OrderbyClause(_) } + def orderbyClause: Parser[OrderbyClause] = ORDER ~ BY ~ rep1sep(orderbyItem, COMMA) ^^ { + case _ ~ _ ~ cols => OrderbyClause(cols) + } def limitClause: Parser[LimitClause] = LIMIT ~> expression ^^ { LimitClause(_) } /** @@ -318,7 +326,7 @@ trait BasicParser extends JavaTokenParsers with Serializable { def combinedClause: Parser[CombinedClause] = selectClause ~ opt(whereClause) ~ opt(groupbyClause) ~ opt(orderbyClause) ~ opt(limitClause) ^^ { case sel ~ whereOpt ~ groupbyOpt ~ orderbyOpt ~ limitOpt => { - val tails = Seq(whereOpt, groupbyOpt, orderbyOpt, limitOpt).flatMap(opt => opt) + val tails = Seq(whereOpt, groupbyOpt, orderbyOpt, limitOpt).flatMap(opt => opt) CombinedClause(sel, tails) } } diff --git a/measure/src/test/resources/config-test-profiling-streaming.json b/measure/src/test/resources/config-test-profiling-streaming.json index 4ad165e6d..b2a74b87e 100644 --- a/measure/src/test/resources/config-test-profiling-streaming.json +++ b/measure/src/test/resources/config-test-profiling-streaming.json @@ -54,7 +54,7 @@ { "dsl.type": "griffin-dsl", "dq.type": "profiling", - "rule": "source.name.count(), source.age.avg(), source.age.max(), source.age.min() group by source.name having source.name.count() > 0 limit 2", + "rule": "source.name.count(), source.age.avg(), source.age.max(), source.age.min() group by source.name", "details": { "source": "source", "profiling": { diff --git a/measure/src/test/resources/config-test-profiling.json b/measure/src/test/resources/config-test-profiling.json index 721dc4752..40aa15775 100644 --- a/measure/src/test/resources/config-test-profiling.json +++ b/measure/src/test/resources/config-test-profiling.json @@ -23,7 +23,7 @@ { "dsl.type": "griffin-dsl", "dq.type": "profiling", - "rule": "source.user_id.count() as cnt", + "rule": "user_id.count() as cnt group by user_id order by cnt desc, user_id desc limit 3", "details": { "source": "source", "profiling": { diff --git a/measure/src/test/scala/org/apache/griffin/measure/process/BatchProcessTest.scala b/measure/src/test/scala/org/apache/griffin/measure/process/BatchProcessTest.scala index 5bb092cfc..a1e4854b4 100644 --- a/measure/src/test/scala/org/apache/griffin/measure/process/BatchProcessTest.scala +++ b/measure/src/test/scala/org/apache/griffin/measure/process/BatchProcessTest.scala @@ -35,8 +35,8 @@ import scala.util.{Failure, Success, Try} class BatchProcessTest extends FunSuite with Matchers with BeforeAndAfter with Loggable { val envFile = "src/test/resources/env-test.json" -// val confFile = "src/test/resources/config-test-profiling.json" - val confFile = "src/test/resources/config-test-accuracy.json" + val confFile = "src/test/resources/config-test-profiling.json" +// val confFile = "src/test/resources/config-test-accuracy.json" val envFsType = "local" val userFsType = "local" diff --git a/measure/src/test/scala/org/apache/griffin/measure/rule/adaptor/GriffinDslAdaptorTest.scala b/measure/src/test/scala/org/apache/griffin/measure/rule/adaptor/GriffinDslAdaptorTest.scala index 85e420f8e..9bf4b1346 100644 --- a/measure/src/test/scala/org/apache/griffin/measure/rule/adaptor/GriffinDslAdaptorTest.scala +++ b/measure/src/test/scala/org/apache/griffin/measure/rule/adaptor/GriffinDslAdaptorTest.scala @@ -36,7 +36,7 @@ class GriffinDslAdaptorTest extends FunSuite with Matchers with BeforeAndAfter w |{ | "dsl.type": "griffin-dsl", | "dq.type": "profiling", - | "rule": "source.user_id.count() as cnt group by source.age having source.desc.count() > 5 order by source.user_id", + | "rule": "source.user_id.count() as cnt group by source.age having source.desc.count() > 5 or false order by user_id desc, user_name asc limit 5", | "details": { | "source": "source", | "profiling": { diff --git a/measure/src/test/scala/org/apache/griffin/measure/rule/dsl/parser/BasicParserTest.scala b/measure/src/test/scala/org/apache/griffin/measure/rule/dsl/parser/BasicParserTest.scala index f70d77186..caa8cf38d 100644 --- a/measure/src/test/scala/org/apache/griffin/measure/rule/dsl/parser/BasicParserTest.scala +++ b/measure/src/test/scala/org/apache/griffin/measure/rule/dsl/parser/BasicParserTest.scala @@ -195,4 +195,11 @@ class BasicParserTest extends FunSuite with Matchers with BeforeAndAfter { result3.get.desc should be ("source.age + 2 * 5 + target.offset * func('a', source.name)") } + test ("order by clause") { + val rule = "order by source.user_id, item" + val result = parser.parseAll(parser.orderbyClause, rule) + result.successful should be (true) + println(result.get.desc) + } + } From 60560ebfe6228fd548209f386eae0095a1da77f7 Mon Sep 17 00:00:00 2001 From: Lionel Liu Date: Fri, 29 Sep 2017 15:52:06 +0800 Subject: [PATCH 088/111] ignore profiling clean step --- .../data/connector/DataConnector.scala | 8 ++++++- .../rule/adaptor/GriffinDslAdaptor.scala | 22 ++++++++++--------- .../test/resources/config-test-profiling.json | 2 +- .../process/StreamingProcessTest.scala | 4 ++-- 4 files changed, 22 insertions(+), 14 deletions(-) diff --git a/measure/src/main/scala/org/apache/griffin/measure/data/connector/DataConnector.scala b/measure/src/main/scala/org/apache/griffin/measure/data/connector/DataConnector.scala index 52d4fcd37..534fb1bc4 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/data/connector/DataConnector.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/data/connector/DataConnector.scala @@ -72,7 +72,13 @@ trait DataConnector extends Loggable with Serializable { val outDf = sqlContext.table(thisTable) // drop temp table - names.foreach(name => sqlContext.dropTempTable(name)) + names.foreach { name => + try { + sqlContext.dropTempTable(name) + } catch { + case e: Throwable => warn(s"drop temp table ${name} fails") + } + } // add tmst val withTmstDf = outDf.withColumn(tmstColName, lit(ms)) diff --git a/measure/src/main/scala/org/apache/griffin/measure/rule/adaptor/GriffinDslAdaptor.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/adaptor/GriffinDslAdaptor.scala index 2efde0651..026324b90 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rule/adaptor/GriffinDslAdaptor.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rule/adaptor/GriffinDslAdaptor.scala @@ -304,17 +304,19 @@ case class GriffinDslAdaptor(dataSourceNames: Seq[String], ) // 2. clear processed data - val clearDataSourceStep = DfOprStep( - s"${sourceName}_clear", - "clear", - Map[String, Any]( - ("df.name" -> sourceName) - ), - NonePersistType, - Some(sourceName) - ) +// val clearDataSourceStep = DfOprStep( +// s"${sourceName}_clear", +// "clear", +// Map[String, Any]( +// ("df.name" -> sourceName) +// ), +// NonePersistType, +// Some(sourceName) +// ) +// +// profilingStep :: clearDataSourceStep :: Nil - profilingStep :: clearDataSourceStep :: Nil + profilingStep:: Nil } } diff --git a/measure/src/test/resources/config-test-profiling.json b/measure/src/test/resources/config-test-profiling.json index 40aa15775..187e88ace 100644 --- a/measure/src/test/resources/config-test-profiling.json +++ b/measure/src/test/resources/config-test-profiling.json @@ -23,7 +23,7 @@ { "dsl.type": "griffin-dsl", "dq.type": "profiling", - "rule": "user_id.count() as cnt group by user_id order by cnt desc, user_id desc limit 3", + "rule": "user_id as id, user_id.approx_count_distinct() as cnt group by user_id order by cnt desc, id desc limit 3", "details": { "source": "source", "profiling": { diff --git a/measure/src/test/scala/org/apache/griffin/measure/process/StreamingProcessTest.scala b/measure/src/test/scala/org/apache/griffin/measure/process/StreamingProcessTest.scala index 73ae06a44..07b7c5e08 100644 --- a/measure/src/test/scala/org/apache/griffin/measure/process/StreamingProcessTest.scala +++ b/measure/src/test/scala/org/apache/griffin/measure/process/StreamingProcessTest.scala @@ -36,8 +36,8 @@ class StreamingProcessTest extends FunSuite with Matchers with BeforeAndAfter wi val envFile = "src/test/resources/env-streaming.json" // val confFile = "src/test/resources/config-test-accuracy-streaming-multids.json" -// val confFile = "src/test/resources/config-test-accuracy-streaming.json" - val confFile = "src/test/resources/config-test-profiling-streaming.json" + val confFile = "src/test/resources/config-test-accuracy-streaming.json" +// val confFile = "src/test/resources/config-test-profiling-streaming.json" val envFsType = "local" val userFsType = "local" From a1c97850592d96b1c29e72d693df1d75055edd3d Mon Sep 17 00:00:00 2001 From: Lionel Liu Date: Fri, 29 Sep 2017 15:54:42 +0800 Subject: [PATCH 089/111] commit --- .../main/scala/org/apache/griffin/measure/utils/HdfsUtil.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/measure/src/main/scala/org/apache/griffin/measure/utils/HdfsUtil.scala b/measure/src/main/scala/org/apache/griffin/measure/utils/HdfsUtil.scala index 69f63beed..9fa6bcfbe 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/utils/HdfsUtil.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/utils/HdfsUtil.scala @@ -28,7 +28,7 @@ object HdfsUtil extends Loggable { private val conf = new Configuration() conf.set("dfs.support.append", "true") - conf.set("fs.defaultFS", "hdfs://localhost") // debug @localhost +// conf.set("fs.defaultFS", "hdfs://localhost") // debug @localhost private val dfs = FileSystem.get(conf) From 4dde3a5df2a6f9c2f7c6636d0eb0c0c664c6732d Mon Sep 17 00:00:00 2001 From: Lionel Liu Date: Fri, 29 Sep 2017 15:58:15 +0800 Subject: [PATCH 090/111] trim --- .../org/apache/griffin/measure/rule/dsl/expr/LogicalExpr.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/measure/src/main/scala/org/apache/griffin/measure/rule/dsl/expr/LogicalExpr.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/dsl/expr/LogicalExpr.scala index 8e28b6763..4b16219e9 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rule/dsl/expr/LogicalExpr.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rule/dsl/expr/LogicalExpr.scala @@ -160,7 +160,7 @@ case class BinaryLogicalExpr(factor: LogicalExpr, tails: Seq[(String, LogicalExp s match { case "&&" => "AND" case "||" => "OR" - case _ => s.toUpperCase + case _ => s.trim.toUpperCase } } override def extractSelf: Expr = { From e7e7ef137b6ce874f95f5848d1f3cec0243750c1 Mon Sep 17 00:00:00 2001 From: Lionel Liu Date: Sat, 30 Sep 2017 14:09:54 +0800 Subject: [PATCH 091/111] version --- measure/derby.log | 13 +++++++++++++ 1 file changed, 13 insertions(+) create mode 100644 measure/derby.log diff --git a/measure/derby.log b/measure/derby.log new file mode 100644 index 000000000..4b93055a3 --- /dev/null +++ b/measure/derby.log @@ -0,0 +1,13 @@ +---------------------------------------------------------------- +Fri Sep 29 15:53:18 CST 2017: +Booting Derby version The Apache Software Foundation - Apache Derby - 10.10.2.0 - (1582446): instance a816c00e-015e-cca0-1a8b-00000f890648 +on database directory /private/var/folders/p0/462y3wrn4lv1fptxx5bwy7b839572r/T/spark-890ab6e2-ee56-4d73-8c6a-0dcce204322e/metastore with class loader sun.misc.Launcher$AppClassLoader@18b4aac2 +Loaded from file:/Users/lliu13/.m2/repository/org/apache/derby/derby/10.10.2.0/derby-10.10.2.0.jar +java.vendor=Oracle Corporation +java.runtime.version=1.8.0_101-b13 +user.dir=/Users/lliu13/git/incubator-griffin/measure +os.name=Mac OS X +os.arch=x86_64 +os.version=10.12.6 +derby.system.home=null +Database Class Loader started - derby.database.classpath='' From 486acdeb7cf23b86aaa757feb4ea7607d69ccbe4 Mon Sep 17 00:00:00 2001 From: Lionel Liu Date: Sat, 30 Sep 2017 15:26:31 +0800 Subject: [PATCH 092/111] add all selection for griffin dsl --- .../measure/rule/adaptor/GriffinDslAdaptor.scala | 10 +++++++++- .../rule/dsl/analyzer/ProfilingAnalyzer.scala | 13 +++++++++++++ .../griffin/measure/rule/dsl/expr/SelectExpr.scala | 11 +++++++++++ .../measure/rule/dsl/parser/BasicParser.scala | 7 +++++-- .../rule/adaptor/GriffinDslAdaptorTest.scala | 2 +- 5 files changed, 39 insertions(+), 4 deletions(-) diff --git a/measure/src/main/scala/org/apache/griffin/measure/rule/adaptor/GriffinDslAdaptor.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/adaptor/GriffinDslAdaptor.scala index 026324b90..2a189d4c3 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rule/adaptor/GriffinDslAdaptor.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rule/adaptor/GriffinDslAdaptor.scala @@ -267,6 +267,8 @@ case class GriffinDslAdaptor(dataSourceNames: Seq[String], } val analyzer = ProfilingAnalyzer(expr.asInstanceOf[ProfilingClause], sourceName) + analyzer.selectionExprs.foreach(println) + val selExprDescs = analyzer.selectionExprs.map { sel => val alias = sel match { case s: AliasableExpr if (s.alias.nonEmpty) => s" AS `${s.alias.get}`" @@ -274,7 +276,13 @@ case class GriffinDslAdaptor(dataSourceNames: Seq[String], } s"${sel.desc}${alias}" } - val selClause = (s"`${GroupByColumn.tmst}`" +: selExprDescs).mkString(", ") + +// val selClause = (s"`${GroupByColumn.tmst}`" +: selExprDescs).mkString(", ") + val selClause = if (analyzer.containsAllSelectionExpr) { + selExprDescs.mkString(", ") + } else { + (s"`${GroupByColumn.tmst}`" +: selExprDescs).mkString(", ") + } // val tailClause = analyzer.tailsExprs.map(_.desc).mkString(" ") val tmstGroupbyClause = GroupbyClause(LiteralStringExpr(s"`${GroupByColumn.tmst}`") :: Nil, None) diff --git a/measure/src/main/scala/org/apache/griffin/measure/rule/dsl/analyzer/ProfilingAnalyzer.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/dsl/analyzer/ProfilingAnalyzer.scala index df1beeed4..34bdbd32d 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rule/dsl/analyzer/ProfilingAnalyzer.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rule/dsl/analyzer/ProfilingAnalyzer.scala @@ -31,6 +31,19 @@ case class ProfilingAnalyzer(expr: ProfilingClause, sourceName: String) extends } val selectionExprs = expr.selectClause.exprs.map(_.extractSelf) + def containsAllSelectionExpr = { + selectionExprs.filter { expr => + expr match { + case SelectionExpr(head: ALLSelectHeadExpr, selectors: Seq[SelectExpr], _) => { + selectors.isEmpty + } + case SelectionExpr(head: DataSourceHeadExpr, selectors: Seq[SelectExpr], _) => { + (head == sourceName) && (selectors.size == 1) && (selectors.head.isInstanceOf[AllFieldsSelectExpr]) + } + case _ => false + } + }.size > 0 + } val groupbyExprOpt = expr.groupbyClauseOpt val preGroupbyExprs = expr.preGroupbyClauses.map(_.extractSelf) diff --git a/measure/src/main/scala/org/apache/griffin/measure/rule/dsl/expr/SelectExpr.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/dsl/expr/SelectExpr.scala index c2c605dbb..fd803a82c 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rule/dsl/expr/SelectExpr.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rule/dsl/expr/SelectExpr.scala @@ -32,6 +32,11 @@ case class FieldNameHeadExpr(field: String) extends HeadExpr { def coalesceDesc: String = desc } +case class ALLSelectHeadExpr() extends HeadExpr { + def desc: String = "*" + def coalesceDesc: String = desc +} + case class OtherHeadExpr(expr: Expr) extends HeadExpr { addChild(expr) @@ -45,6 +50,12 @@ case class OtherHeadExpr(expr: Expr) extends HeadExpr { trait SelectExpr extends Expr with AliasableExpr { } +case class AllFieldsSelectExpr() extends SelectExpr { + def desc: String = s".*" + def coalesceDesc: String = desc + def alias: Option[String] = None +} + case class FieldSelectExpr(field: String) extends SelectExpr { def desc: String = s".${field}" def coalesceDesc: String = desc diff --git a/measure/src/main/scala/org/apache/griffin/measure/rule/dsl/parser/BasicParser.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/dsl/parser/BasicParser.scala index 12d1f0cad..ce4ec6948 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rule/dsl/parser/BasicParser.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rule/dsl/parser/BasicParser.scala @@ -80,7 +80,7 @@ trait BasicParser extends JavaTokenParsers with Serializable { protected def genNamesParser(names: Seq[String]): Parser[String] = { names.reverse.map { - fn => s"""${fn}""": Parser[String] + fn => s"""(?i)${fn}""".r: Parser[String] }.reduce(_ | _) } @@ -183,8 +183,11 @@ trait BasicParser extends JavaTokenParsers with Serializable { OtherHeadExpr(_) } | TableFieldName ^^ { FieldNameHeadExpr(_) + } | ALLSL ^^ { _ => + ALLSelectHeadExpr() } - def selector: Parser[SelectExpr] = functionSelect | fieldSelect | indexSelect + def selector: Parser[SelectExpr] = functionSelect | allFieldsSelect | fieldSelect | indexSelect + def allFieldsSelect: Parser[AllFieldsSelectExpr] = DOT ~> ALLSL ^^ { _ => AllFieldsSelectExpr() } def fieldSelect: Parser[FieldSelectExpr] = DOT ~> TableFieldName ^^ { FieldSelectExpr(_) } def indexSelect: Parser[IndexSelectExpr] = LSQBR ~> argument <~ RSQBR ^^ { IndexSelectExpr(_) } def functionSelect: Parser[FunctionSelectExpr] = DOT ~ FunctionName ~ LBR ~ repsep(argument, COMMA) ~ RBR ^^ { diff --git a/measure/src/test/scala/org/apache/griffin/measure/rule/adaptor/GriffinDslAdaptorTest.scala b/measure/src/test/scala/org/apache/griffin/measure/rule/adaptor/GriffinDslAdaptorTest.scala index 9bf4b1346..2efd8bdce 100644 --- a/measure/src/test/scala/org/apache/griffin/measure/rule/adaptor/GriffinDslAdaptorTest.scala +++ b/measure/src/test/scala/org/apache/griffin/measure/rule/adaptor/GriffinDslAdaptorTest.scala @@ -36,7 +36,7 @@ class GriffinDslAdaptorTest extends FunSuite with Matchers with BeforeAndAfter w |{ | "dsl.type": "griffin-dsl", | "dq.type": "profiling", - | "rule": "source.user_id.count() as cnt group by source.age having source.desc.count() > 5 or false order by user_id desc, user_name asc limit 5", + | "rule": "source.age, source.user_id.COUNT() as cnt group by source.age having source.desc.count() > 5 or false order by user_id desc, user_name asc limit 5", | "details": { | "source": "source", | "profiling": { From 5e45893665e8db091dab8ddbba4698c6237eb8d0 Mon Sep 17 00:00:00 2001 From: Lionel Liu Date: Sat, 30 Sep 2017 16:26:57 +0800 Subject: [PATCH 093/111] doc for dsl --- .../griffin/measure/rule/dsl/parser/BasicParser.scala | 2 +- .../measure/rule/adaptor/GriffinDslAdaptorTest.scala | 2 +- .../griffin/measure/rule/dsl/parser/BasicParserTest.scala | 8 ++++---- 3 files changed, 6 insertions(+), 6 deletions(-) diff --git a/measure/src/main/scala/org/apache/griffin/measure/rule/dsl/parser/BasicParser.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/dsl/parser/BasicParser.scala index ce4ec6948..0431354a2 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rule/dsl/parser/BasicParser.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rule/dsl/parser/BasicParser.scala @@ -131,7 +131,7 @@ trait BasicParser extends JavaTokenParsers with Serializable { import Operator._ object Strings { - def AnyString: Parser[String] = """"(?:[^\"]|[\\][\"])*"""".r | """'(?:[^']|[\\]['])*'""".r + def AnyString: Parser[String] = """"(?:[^\"]|\")*"""".r | """'(?:[^']|\')*'""".r def UQuoteTableFieldName: Parser[String] = """`(?:[^`]|[\\][`])*`""".r def TableFieldName: Parser[String] = UQuoteTableFieldName | """[a-zA-Z_]\w*""".r def DataSourceName: Parser[String] = genNamesParser(dataSourceNames) diff --git a/measure/src/test/scala/org/apache/griffin/measure/rule/adaptor/GriffinDslAdaptorTest.scala b/measure/src/test/scala/org/apache/griffin/measure/rule/adaptor/GriffinDslAdaptorTest.scala index 2efd8bdce..987a06051 100644 --- a/measure/src/test/scala/org/apache/griffin/measure/rule/adaptor/GriffinDslAdaptorTest.scala +++ b/measure/src/test/scala/org/apache/griffin/measure/rule/adaptor/GriffinDslAdaptorTest.scala @@ -36,7 +36,7 @@ class GriffinDslAdaptorTest extends FunSuite with Matchers with BeforeAndAfter w |{ | "dsl.type": "griffin-dsl", | "dq.type": "profiling", - | "rule": "source.age, source.user_id.COUNT() as cnt group by source.age having source.desc.count() > 5 or false order by user_id desc, user_name asc limit 5", + | "rule": "source.age, (source.user_id.COUNT() + 1s) as cnt group by source.age having source.desc.count() > 5 or false order by user_id desc, user_name asc limit 5", | "details": { | "source": "source", | "profiling": { diff --git a/measure/src/test/scala/org/apache/griffin/measure/rule/dsl/parser/BasicParserTest.scala b/measure/src/test/scala/org/apache/griffin/measure/rule/dsl/parser/BasicParserTest.scala index caa8cf38d..d8c95313d 100644 --- a/measure/src/test/scala/org/apache/griffin/measure/rule/dsl/parser/BasicParserTest.scala +++ b/measure/src/test/scala/org/apache/griffin/measure/rule/dsl/parser/BasicParserTest.scala @@ -45,15 +45,15 @@ class BasicParserTest extends FunSuite with Matchers with BeforeAndAfter { result2.successful should be (true) result2.get.desc should be ("NaN") - val rule3 = """'testing'""" + val rule3 = """'test\'ing'""" val result3 = parser.parseAll(parser.literal, rule3) result3.successful should be (true) - result3.get.desc should be ("'testing'") + result3.get.desc should be ("'test\\'ing'") - val rule4 = """"test again"""" + val rule4 = """"test\" again"""" val result4 = parser.parseAll(parser.literal, rule4) result4.successful should be (true) - result4.get.desc should be ("\"test again\"") + result4.get.desc should be ("\"test\\\" again\"") val rule5 = """-1.342""" val result5 = parser.parseAll(parser.literal, rule5) From 7f2386a7dbe8d38eae717d6f4ffec1da0b584610 Mon Sep 17 00:00:00 2001 From: Lionel Liu Date: Sat, 30 Sep 2017 16:27:19 +0800 Subject: [PATCH 094/111] doc for dsl --- griffin-doc/dsl-guide.md | 83 ++++++++++++++++++++++++++++++++++++++++ 1 file changed, 83 insertions(+) create mode 100644 griffin-doc/dsl-guide.md diff --git a/griffin-doc/dsl-guide.md b/griffin-doc/dsl-guide.md new file mode 100644 index 000000000..6a7b3f8e5 --- /dev/null +++ b/griffin-doc/dsl-guide.md @@ -0,0 +1,83 @@ + +# Apache Griffin DSL Guide +Griffin DSL is designed for DQ measurement, as a SQL-like language, trying to describe the DQ domain request. + +## Griffin DSL Syntax Description +Griffin DSL is SQL-like, case insensitive, and easy to learn. + +### Supporting process +- logical operation: not, and, or, in, between, like, 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` +- `as, where, group, by, having, order, desc, asc, limit` + +### Operators +- `!, &&, ||, =, !=, <, >, <=, >=, <>` +- `+, -, *, /, %` +- `(, )` +- `., [, ]` + +### Literals +- **string**: any string surrounded with a pair of " or ', with escape charactor \ if any request. + e.g. `"test", 'string 1', "hello \" world \" "` +- **number**: double or integer number. + e.g. `123, 33.5` +- **time**: a integer with unit in a string, will be translated to a integer number in millisecond. + e.g. `3d, 5h, 4ms` +- **boolean**: boolean value directly. + e.g. `true, false` + +### Selections +- **selection head**: data source name. + e.g. `source, target` +- **all field selection**: * or with data source name ahead. + e.g. `*, source.*, target.*` +- **field selection**: field name or with data source name ahead. + e.g. `source.age, target.name, user_id` +- **index selection**: interget between square brackets "[]" with field name ahead. + e.g. `source.attributes[3]` +- **function selection**: function name with brackets "()", with field name ahead or not. + e.g. `count(*), *.count(), source.user_id.count(), max(source.age)` +- **alias**: declare an alias after a selection. + e.g. `source.user_id as id, target.user_name as name` + +### Math expressions +- **math factor**: literal or function or selection or math exression with brackets. + e.g. `123, max(1, 2, 3, 4), source.age, (source.age + 13)` +- **unary math expression**: unary math operator with factor. + e.g. `-(100 - source.score)` +- **binary math expression**: math factors with binary math operators. + e.g. `source.age + 13, score * 2 + ratio` + +### Logical expression +- **in**: in clause like sql. + e.g. `source.country in ("USA", "CHN", "RSA")` +- **between**: between clause like sql. + e.g. `source.age between 3 and 30, source.age between (3, 30)` +- **like**: like clause like sql. + e.g. `source.name like "%abc%"` +- **logical factor**: + From eae139c33b81bedf61ad46ab1c77c7199ca5497e Mon Sep 17 00:00:00 2001 From: Lionel Liu Date: Mon, 9 Oct 2017 15:11:13 +0800 Subject: [PATCH 095/111] doc --- griffin-doc/dsl-guide.md | 22 +++++++++++++++++++++- 1 file changed, 21 insertions(+), 1 deletion(-) diff --git a/griffin-doc/dsl-guide.md b/griffin-doc/dsl-guide.md index 6a7b3f8e5..40a37a435 100644 --- a/griffin-doc/dsl-guide.md +++ b/griffin-doc/dsl-guide.md @@ -79,5 +79,25 @@ Griffin DSL is SQL-like, case insensitive, and easy to learn. e.g. `source.age between 3 and 30, source.age between (3, 30)` - **like**: like clause like sql. e.g. `source.name like "%abc%"` -- **logical factor**: +- **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` + e.g. `source.age is not nan` +- **logical factor**: math expression or logical expressions above or other logical expressions with brackets. + e.g. `(source.user_id = target.user_id AND source.age > target.age)` +- **unary logical expression**: unary logical operator with factor. + e.g. `NOT source.has_data` +- **binary logical expression**: logical factors with binary logical operators, including `and`, `or` and comparison operators. + e.g. `source.age = target.age OR source.ticket = target.tck` + +### Expression +- **expression**: logical expression and math expression. + +### Function +- **argument**: expression. +- **function**: function name with arguments between brackets. + e.g. `max(source.age, target.age), count(*)` + +### Clause +- **select clause**: \ No newline at end of file From 3e3ec519c38a676a7a99cef189682bf531029adc Mon Sep 17 00:00:00 2001 From: Lionel Liu Date: Tue, 10 Oct 2017 14:22:34 +0800 Subject: [PATCH 096/111] dsl support select and from --- measure/derby.log | 13 - .../rule/adaptor/GriffinDslAdaptor.scala | 20 +- .../rule/dsl/expr/ClauseExpression.scala | 46 +- .../measure/rule/dsl/parser/BasicParser.scala | 21 +- .../rule/dsl/parser/GriffinDslParser.scala | 8 +- .../test/resources/config-test-profiling.json | 3 +- .../measure/process/BatchProcessTest.scala | 292 +++--- .../measure/process/JsonParseTest.scala | 980 +++++++++--------- .../process/StreamingProcessTest.scala | 294 +++--- .../rule/dsl/parser/BasicParserTest.scala | 14 + 10 files changed, 867 insertions(+), 824 deletions(-) delete mode 100644 measure/derby.log diff --git a/measure/derby.log b/measure/derby.log deleted file mode 100644 index 4b93055a3..000000000 --- a/measure/derby.log +++ /dev/null @@ -1,13 +0,0 @@ ----------------------------------------------------------------- -Fri Sep 29 15:53:18 CST 2017: -Booting Derby version The Apache Software Foundation - Apache Derby - 10.10.2.0 - (1582446): instance a816c00e-015e-cca0-1a8b-00000f890648 -on database directory /private/var/folders/p0/462y3wrn4lv1fptxx5bwy7b839572r/T/spark-890ab6e2-ee56-4d73-8c6a-0dcce204322e/metastore with class loader sun.misc.Launcher$AppClassLoader@18b4aac2 -Loaded from file:/Users/lliu13/.m2/repository/org/apache/derby/derby/10.10.2.0/derby-10.10.2.0.jar -java.vendor=Oracle Corporation -java.runtime.version=1.8.0_101-b13 -user.dir=/Users/lliu13/git/incubator-griffin/measure -os.name=Mac OS X -os.arch=x86_64 -os.version=10.12.6 -derby.system.home=null -Database Class Loader started - derby.database.classpath='' diff --git a/measure/src/main/scala/org/apache/griffin/measure/rule/adaptor/GriffinDslAdaptor.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/adaptor/GriffinDslAdaptor.scala index 2a189d4c3..8199d80a1 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rule/adaptor/GriffinDslAdaptor.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rule/adaptor/GriffinDslAdaptor.scala @@ -261,13 +261,19 @@ case class GriffinDslAdaptor(dataSourceNames: Seq[String], } } case ProfilingType => { - val sourceName = getNameOpt(details, ProfilingInfo._Source) match { - case Some(name) => name - case _ => dataSourceNames.head + val profilingClause = expr.asInstanceOf[ProfilingClause] + val sourceName = profilingClause.fromClauseOpt match { + case Some(fc) => fc.dataSource + case _ => { + getNameOpt(details, ProfilingInfo._Source) match { + case Some(name) => name + case _ => dataSourceNames.head + } + } } - val analyzer = ProfilingAnalyzer(expr.asInstanceOf[ProfilingClause], sourceName) + val analyzer = ProfilingAnalyzer(profilingClause, sourceName) - analyzer.selectionExprs.foreach(println) +// analyzer.selectionExprs.foreach(println) val selExprDescs = analyzer.selectionExprs.map { sel => val alias = sel match { @@ -284,6 +290,8 @@ case class GriffinDslAdaptor(dataSourceNames: Seq[String], (s"`${GroupByColumn.tmst}`" +: selExprDescs).mkString(", ") } + val fromClause = profilingClause.fromClauseOpt.getOrElse(FromClause(sourceName)).desc + // val tailClause = analyzer.tailsExprs.map(_.desc).mkString(" ") val tmstGroupbyClause = GroupbyClause(LiteralStringExpr(s"`${GroupByColumn.tmst}`") :: Nil, None) val mergedGroubbyClause = tmstGroupbyClause.merge(analyzer.groupbyExprOpt match { @@ -300,7 +308,7 @@ case class GriffinDslAdaptor(dataSourceNames: Seq[String], // 1. select statement val profilingSql = { // s"SELECT `${GroupByColumn.tmst}`, ${selClause} FROM ${sourceName} ${tailClause} GROUP BY `${GroupByColumn.tmst}`" - s"SELECT ${selClause} FROM ${sourceName} ${preGroupbyClause} ${groupbyClause} ${postGroupbyClause}" + s"SELECT ${selClause} ${fromClause} ${preGroupbyClause} ${groupbyClause} ${postGroupbyClause}" } val profilingMetricName = resultName(details, ProfilingInfo._Profiling) val profilingStep = SparkSqlStep( diff --git a/measure/src/main/scala/org/apache/griffin/measure/rule/dsl/expr/ClauseExpression.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/dsl/expr/ClauseExpression.scala index 26882b493..c0986e1a6 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rule/dsl/expr/ClauseExpression.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rule/dsl/expr/ClauseExpression.scala @@ -26,7 +26,14 @@ case class SelectClause(exprs: Seq[Expr]) extends ClauseExpression { addChildren(exprs) def desc: String = s"${exprs.map(_.desc).mkString(", ")}" - def coalesceDesc: String = s"${exprs.map(_.desc).mkString(", ")}" + def coalesceDesc: String = desc + +} + +case class FromClause(dataSource: String) extends ClauseExpression { + + def desc: String = s"FROM `${dataSource}`" + def coalesceDesc: String = desc } @@ -107,44 +114,61 @@ case class LimitClause(expr: Expr) extends ClauseExpression { def coalesceDesc: String = s"LIMIT ${expr.coalesceDesc}" } -case class CombinedClause(selectClause: SelectClause, tails: Seq[ClauseExpression] +case class CombinedClause(selectClause: SelectClause, fromClauseOpt: Option[FromClause], + tails: Seq[ClauseExpression] ) extends ClauseExpression { - addChildren(selectClause +: tails) + addChildren({ + val headClauses: Seq[ClauseExpression] = selectClause +: (fromClauseOpt.toSeq) + headClauses ++ tails + }) def desc: String = { - tails.foldLeft(selectClause.desc) { (head, tail) => + val selectDesc = s"SELECT ${selectClause.desc}" + val fromDesc = fromClauseOpt.map(_.desc).mkString(" ") + val headDesc = s"${selectDesc} ${fromDesc}" + tails.foldLeft(headDesc) { (head, tail) => s"${head} ${tail.desc}" } } def coalesceDesc: String = { - tails.foldLeft(selectClause.coalesceDesc) { (head, tail) => + val selectDesc = s"SELECT ${selectClause.coalesceDesc}" + val fromDesc = fromClauseOpt.map(_.coalesceDesc).mkString(" ") + val headDesc = s"${selectDesc} ${fromDesc}" + tails.foldLeft(headDesc) { (head, tail) => s"${head} ${tail.coalesceDesc}" } } } -case class ProfilingClause(selectClause: SelectClause, groupbyClauseOpt: Option[GroupbyClause], +case class ProfilingClause(selectClause: SelectClause, + fromClauseOpt: Option[FromClause], + groupbyClauseOpt: Option[GroupbyClause], preGroupbyClauses: Seq[ClauseExpression], postGroupbyClauses: Seq[ClauseExpression] ) extends ClauseExpression { - addChildren(groupbyClauseOpt match { - case Some(gc) => (selectClause +: preGroupbyClauses) ++ (gc +: postGroupbyClauses) - case _ => (selectClause +: preGroupbyClauses) ++ postGroupbyClauses + addChildren({ + val headClauses: Seq[ClauseExpression] = selectClause +: (fromClauseOpt.toSeq) + groupbyClauseOpt match { + case Some(gc) => (headClauses ++ preGroupbyClauses) ++ (gc +: postGroupbyClauses) + case _ => (headClauses ++ preGroupbyClauses) ++ postGroupbyClauses + } }) def desc: String = { val selectDesc = selectClause.desc + val fromDesc = fromClauseOpt.map(_.desc).mkString(" ") val groupbyDesc = groupbyClauseOpt.map(_.desc).mkString(" ") val preDesc = preGroupbyClauses.map(_.desc).mkString(" ") val postDesc = postGroupbyClauses.map(_.desc).mkString(" ") - s"${selectDesc} ${preDesc} ${groupbyDesc} ${postDesc}" + s"${selectDesc} ${fromDesc} ${preDesc} ${groupbyDesc} ${postDesc}" } def coalesceDesc: String = { val selectDesc = selectClause.coalesceDesc + val fromDesc = fromClauseOpt.map(_.coalesceDesc).mkString(" ") val groupbyDesc = groupbyClauseOpt.map(_.coalesceDesc).mkString(" ") val preDesc = preGroupbyClauses.map(_.coalesceDesc).mkString(" ") val postDesc = postGroupbyClauses.map(_.coalesceDesc).mkString(" ") - s"${selectDesc} ${preDesc} ${groupbyDesc} ${postDesc}" + s"${selectDesc} ${fromDesc} ${preDesc} ${groupbyDesc} ${postDesc}" } } \ No newline at end of file diff --git a/measure/src/main/scala/org/apache/griffin/measure/rule/dsl/parser/BasicParser.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/dsl/parser/BasicParser.scala index 0431354a2..f55b1f84b 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rule/dsl/parser/BasicParser.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rule/dsl/parser/BasicParser.scala @@ -27,6 +27,14 @@ trait BasicParser extends JavaTokenParsers with Serializable { val dataSourceNames: Seq[String] val functionNames: Seq[String] + private def trim(str: String): String = { + val regex = """`(.*)`""".r + str match { + case regex(s) => s + case _ => str + } + } + /** * BNF for basic parser * @@ -118,6 +126,8 @@ trait BasicParser extends JavaTokenParsers with Serializable { val UQUOTE: Parser[String] = "`" val COMMA: Parser[String] = "," + val SELECT: Parser[String] = """(?i)select\s""".r + val FROM: Parser[String] = """(?i)from\s""".r val AS: Parser[String] = """(?i)as\s""".r val WHERE: Parser[String] = """(?i)where\s""".r val GROUP: Parser[String] = """(?i)group\s""".r @@ -307,7 +317,8 @@ trait BasicParser extends JavaTokenParsers with Serializable { * = */ - def selectClause: Parser[SelectClause] = rep1sep(expression, COMMA) ^^ { SelectClause(_) } + def selectClause: Parser[SelectClause] = opt(SELECT) ~> rep1sep(expression, COMMA) ^^ { SelectClause(_) } + def fromClause: Parser[FromClause] = FROM ~> TableFieldName ^^ { ds => FromClause(trim(ds)) } def whereClause: Parser[WhereClause] = WHERE ~> expression ^^ { WhereClause(_) } def havingClause: Parser[Expr] = HAVING ~> expression def groupbyClause: Parser[GroupbyClause] = GROUP ~ BY ~ rep1sep(expression, COMMA) ~ opt(havingClause) ^^ { @@ -323,14 +334,14 @@ trait BasicParser extends JavaTokenParsers with Serializable { /** * -- combined clauses -- - * = [ ]+ [ ]+ [ ]+ [ ]+ + * = [ ]+ [ ]+ [ ]+ [ ]+ [ ]+ */ - def combinedClause: Parser[CombinedClause] = selectClause ~ opt(whereClause) ~ + def combinedClause: Parser[CombinedClause] = selectClause ~ opt(fromClause) ~ opt(whereClause) ~ opt(groupbyClause) ~ opt(orderbyClause) ~ opt(limitClause) ^^ { - case sel ~ whereOpt ~ groupbyOpt ~ orderbyOpt ~ limitOpt => { + case sel ~ fromOpt ~ whereOpt ~ groupbyOpt ~ orderbyOpt ~ limitOpt => { val tails = Seq(whereOpt, groupbyOpt, orderbyOpt, limitOpt).flatMap(opt => opt) - CombinedClause(sel, tails) + CombinedClause(sel, fromOpt, tails) } } diff --git a/measure/src/main/scala/org/apache/griffin/measure/rule/dsl/parser/GriffinDslParser.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/dsl/parser/GriffinDslParser.scala index 637decbff..0800f4571 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rule/dsl/parser/GriffinDslParser.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rule/dsl/parser/GriffinDslParser.scala @@ -26,15 +26,15 @@ case class GriffinDslParser(dataSourceNames: Seq[String], functionNames: Seq[Str /** * -- profiling clauses -- - * = [ ]+ [ ]+ [ ]+ [ ]+ + * = [ ]+ [ ]+ [ ]+ [ ]+ [ ]+ */ - def profilingClause: Parser[ProfilingClause] = selectClause ~ opt(whereClause) ~ + def profilingClause: Parser[ProfilingClause] = selectClause ~ opt(fromClause) ~ opt(whereClause) ~ opt(groupbyClause) ~ opt(orderbyClause) ~ opt(limitClause) ^^ { - case sel ~ whereOpt ~ groupbyOpt ~ orderbyOpt ~ limitOpt => { + case sel ~ fromOpt ~ whereOpt ~ groupbyOpt ~ orderbyOpt ~ limitOpt => { val preClauses = Seq(whereOpt).flatMap(opt => opt) val postClauses = Seq(orderbyOpt, limitOpt).flatMap(opt => opt) - ProfilingClause(sel, groupbyOpt, preClauses, postClauses) + ProfilingClause(sel, fromOpt, groupbyOpt, preClauses, postClauses) } } diff --git a/measure/src/test/resources/config-test-profiling.json b/measure/src/test/resources/config-test-profiling.json index 187e88ace..7c16f24a9 100644 --- a/measure/src/test/resources/config-test-profiling.json +++ b/measure/src/test/resources/config-test-profiling.json @@ -23,9 +23,8 @@ { "dsl.type": "griffin-dsl", "dq.type": "profiling", - "rule": "user_id as id, user_id.approx_count_distinct() as cnt group by user_id order by cnt desc, id desc limit 3", + "rule": "select user_id as id, user_id.count() as cnt from source group by user_id order by cnt desc, id desc limit 3", "details": { - "source": "source", "profiling": { "name": "count", "persist.type": "metric" diff --git a/measure/src/test/scala/org/apache/griffin/measure/process/BatchProcessTest.scala b/measure/src/test/scala/org/apache/griffin/measure/process/BatchProcessTest.scala index a1e4854b4..845a05178 100644 --- a/measure/src/test/scala/org/apache/griffin/measure/process/BatchProcessTest.scala +++ b/measure/src/test/scala/org/apache/griffin/measure/process/BatchProcessTest.scala @@ -1,146 +1,146 @@ -/* -Licensed to the Apache Software Foundation (ASF) under one -or more contributor license agreements. See the NOTICE file -distributed with this work for additional information -regarding copyright ownership. The ASF licenses this file -to you under the Apache License, Version 2.0 (the -"License"); you may not use this file except in compliance -with the License. You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, -software distributed under the License is distributed on an -"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -KIND, either express or implied. See the License for the -specific language governing permissions and limitations -under the License. -*/ -package org.apache.griffin.measure.process - -import org.apache.griffin.measure.config.params.env._ -import org.apache.griffin.measure.config.params.user._ -import org.apache.griffin.measure.config.params._ -import org.apache.griffin.measure.config.reader.ParamReaderFactory -import org.apache.griffin.measure.config.validator.AllParamValidator -import org.apache.griffin.measure.log.Loggable -import org.apache.griffin.measure.persist.PersistThreadPool -import org.junit.runner.RunWith -import org.scalatest.junit.JUnitRunner -import org.scalatest.{BeforeAndAfter, FunSuite, Matchers} - -import scala.util.{Failure, Success, Try} - -@RunWith(classOf[JUnitRunner]) -class BatchProcessTest extends FunSuite with Matchers with BeforeAndAfter with Loggable { - - val envFile = "src/test/resources/env-test.json" - val confFile = "src/test/resources/config-test-profiling.json" -// val confFile = "src/test/resources/config-test-accuracy.json" - - val envFsType = "local" - val userFsType = "local" - - val args = Array(envFile, confFile) - - var allParam: AllParam = _ - - before { - // read param files - val envParam = readParamFile[EnvParam](envFile, envFsType) match { - case Success(p) => p - case Failure(ex) => { - error(ex.getMessage) - sys.exit(-2) - } - } - val userParam = readParamFile[UserParam](confFile, userFsType) match { - case Success(p) => p - case Failure(ex) => { - error(ex.getMessage) - sys.exit(-2) - } - } - allParam = AllParam(envParam, userParam) - - // validate param files - validateParams(allParam) match { - case Failure(ex) => { - error(ex.getMessage) - sys.exit(-3) - } - case _ => { - info("params validation pass") - } - } - } - - test ("batch process") { - val procType = ProcessType(allParam.userParam.procType) - val proc: DqProcess = procType match { - case BatchProcessType => BatchDqProcess(allParam) - case StreamingProcessType => StreamingDqProcess(allParam) - case _ => { - error(s"${procType} is unsupported process type!") - sys.exit(-4) - } - } - - // process init - proc.init match { - case Success(_) => { - info("process init success") - } - case Failure(ex) => { - error(s"process init error: ${ex.getMessage}") - shutdown - sys.exit(-5) - } - } - - // process run - proc.run match { - case Success(_) => { - info("process run success") - } - case Failure(ex) => { - error(s"process run error: ${ex.getMessage}") - - if (proc.retriable) { - throw ex - } else { - shutdown - sys.exit(-5) - } - } - } - - // process end - proc.end match { - case Success(_) => { - info("process end success") - } - case Failure(ex) => { - error(s"process end error: ${ex.getMessage}") - shutdown - sys.exit(-5) - } - } - - shutdown - } - - private def readParamFile[T <: Param](file: String, fsType: String)(implicit m : Manifest[T]): Try[T] = { - val paramReader = ParamReaderFactory.getParamReader(file, fsType) - paramReader.readConfig[T] - } - - private def validateParams(allParam: AllParam): Try[Boolean] = { - val allParamValidator = AllParamValidator() - allParamValidator.validate(allParam) - } - - private def shutdown(): Unit = { - PersistThreadPool.shutdown - } -} +///* +//Licensed to the Apache Software Foundation (ASF) under one +//or more contributor license agreements. See the NOTICE file +//distributed with this work for additional information +//regarding copyright ownership. The ASF licenses this file +//to you under the Apache License, Version 2.0 (the +//"License"); you may not use this file except in compliance +//with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +//Unless required by applicable law or agreed to in writing, +//software distributed under the License is distributed on an +//"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +//KIND, either express or implied. See the License for the +//specific language governing permissions and limitations +//under the License. +//*/ +//package org.apache.griffin.measure.process +// +//import org.apache.griffin.measure.config.params.env._ +//import org.apache.griffin.measure.config.params.user._ +//import org.apache.griffin.measure.config.params._ +//import org.apache.griffin.measure.config.reader.ParamReaderFactory +//import org.apache.griffin.measure.config.validator.AllParamValidator +//import org.apache.griffin.measure.log.Loggable +//import org.apache.griffin.measure.persist.PersistThreadPool +//import org.junit.runner.RunWith +//import org.scalatest.junit.JUnitRunner +//import org.scalatest.{BeforeAndAfter, FunSuite, Matchers} +// +//import scala.util.{Failure, Success, Try} +// +//@RunWith(classOf[JUnitRunner]) +//class BatchProcessTest extends FunSuite with Matchers with BeforeAndAfter with Loggable { +// +// val envFile = "src/test/resources/env-test.json" +// val confFile = "src/test/resources/config-test-profiling.json" +//// val confFile = "src/test/resources/config-test-accuracy.json" +// +// val envFsType = "local" +// val userFsType = "local" +// +// val args = Array(envFile, confFile) +// +// var allParam: AllParam = _ +// +// before { +// // read param files +// val envParam = readParamFile[EnvParam](envFile, envFsType) match { +// case Success(p) => p +// case Failure(ex) => { +// error(ex.getMessage) +// sys.exit(-2) +// } +// } +// val userParam = readParamFile[UserParam](confFile, userFsType) match { +// case Success(p) => p +// case Failure(ex) => { +// error(ex.getMessage) +// sys.exit(-2) +// } +// } +// allParam = AllParam(envParam, userParam) +// +// // validate param files +// validateParams(allParam) match { +// case Failure(ex) => { +// error(ex.getMessage) +// sys.exit(-3) +// } +// case _ => { +// info("params validation pass") +// } +// } +// } +// +// test ("batch process") { +// val procType = ProcessType(allParam.userParam.procType) +// val proc: DqProcess = procType match { +// case BatchProcessType => BatchDqProcess(allParam) +// case StreamingProcessType => StreamingDqProcess(allParam) +// case _ => { +// error(s"${procType} is unsupported process type!") +// sys.exit(-4) +// } +// } +// +// // process init +// proc.init match { +// case Success(_) => { +// info("process init success") +// } +// case Failure(ex) => { +// error(s"process init error: ${ex.getMessage}") +// shutdown +// sys.exit(-5) +// } +// } +// +// // process run +// proc.run match { +// case Success(_) => { +// info("process run success") +// } +// case Failure(ex) => { +// error(s"process run error: ${ex.getMessage}") +// +// if (proc.retriable) { +// throw ex +// } else { +// shutdown +// sys.exit(-5) +// } +// } +// } +// +// // process end +// proc.end match { +// case Success(_) => { +// info("process end success") +// } +// case Failure(ex) => { +// error(s"process end error: ${ex.getMessage}") +// shutdown +// sys.exit(-5) +// } +// } +// +// shutdown +// } +// +// private def readParamFile[T <: Param](file: String, fsType: String)(implicit m : Manifest[T]): Try[T] = { +// val paramReader = ParamReaderFactory.getParamReader(file, fsType) +// paramReader.readConfig[T] +// } +// +// private def validateParams(allParam: AllParam): Try[Boolean] = { +// val allParamValidator = AllParamValidator() +// allParamValidator.validate(allParam) +// } +// +// private def shutdown(): Unit = { +// PersistThreadPool.shutdown +// } +//} diff --git a/measure/src/test/scala/org/apache/griffin/measure/process/JsonParseTest.scala b/measure/src/test/scala/org/apache/griffin/measure/process/JsonParseTest.scala index b119d7650..1273bcf97 100644 --- a/measure/src/test/scala/org/apache/griffin/measure/process/JsonParseTest.scala +++ b/measure/src/test/scala/org/apache/griffin/measure/process/JsonParseTest.scala @@ -1,91 +1,249 @@ -/* -Licensed to the Apache Software Foundation (ASF) under one -or more contributor license agreements. See the NOTICE file -distributed with this work for additional information -regarding copyright ownership. The ASF licenses this file -to you under the Apache License, Version 2.0 (the -"License"); you may not use this file except in compliance -with the License. You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, -software distributed under the License is distributed on an -"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -KIND, either express or implied. See the License for the -specific language governing permissions and limitations -under the License. -*/ -package org.apache.griffin.measure.process - -import org.apache.griffin.measure.config.params._ -import org.apache.griffin.measure.config.params.env._ -import org.apache.griffin.measure.config.params.user._ -import org.apache.griffin.measure.config.reader.ParamReaderFactory -import org.apache.griffin.measure.config.validator.AllParamValidator -import org.apache.griffin.measure.log.Loggable -import org.apache.griffin.measure.persist.PersistThreadPool -import org.apache.griffin.measure.process.engine.DataFrameOprs -import org.apache.griffin.measure.utils.{HdfsUtil, JsonUtil} -import org.apache.hadoop.hive.ql.exec.UDF -import org.apache.spark.{SparkConf, SparkContext} -import org.apache.spark.sql._ -import org.apache.spark.sql.catalyst.expressions.GenericRowWithSchema -import org.apache.spark.sql.expressions.UserDefinedAggregateFunction -import org.apache.spark.sql.hive.HiveContext -import org.apache.spark.sql.types._ -import org.junit.runner.RunWith -import org.scalatest.junit.JUnitRunner -import org.scalatest.{BeforeAndAfter, FunSuite, Matchers} - -import scala.collection.mutable.WrappedArray -import scala.util.{Failure, Success, Try} - -@RunWith(classOf[JUnitRunner]) -class JsonParseTest extends FunSuite with Matchers with BeforeAndAfter with Loggable { - - var sparkContext: SparkContext = _ - var sqlContext: SQLContext = _ - - before { - val conf = new SparkConf().setAppName("test json").setMaster("local[*]") - sparkContext = new SparkContext(conf) - sparkContext.setLogLevel("WARN") -// sqlContext = new HiveContext(sparkContext) - sqlContext = new SQLContext(sparkContext) - } - - test ("json test") { - // 0. prepare data -// val dt = -// """ -// |{"name": "s1", "age": 12, "items": [1, 2, 3], -// |"subs": [{"id": 1, "type": "seed"}, {"id": 2, "type": "frog"}], -// |"inner": {"a": 1, "b": 2}, "jstr": "{\"s1\": \"aaa\", \"s2\": 123}" -// |}""".stripMargin -// val rdd0 = sparkContext.parallelize(Seq(dt)).map(Row(_)) - val rdd0 = sparkContext.textFile("src/test/resources/input.msg").map(Row(_)) - - val vtp = StructField("value", StringType) - val df0 = sqlContext.createDataFrame(rdd0, StructType(Array(vtp))) - df0.registerTempTable("src") - -// val fromJson2Array = (s: String) => { -// JsonUtil.fromJson[Seq[String]](s) -// } -// sqlContext.udf.register("from_json_to_array", fromJson2Array) +///* +//Licensed to the Apache Software Foundation (ASF) under one +//or more contributor license agreements. See the NOTICE file +//distributed with this work for additional information +//regarding copyright ownership. The ASF licenses this file +//to you under the Apache License, Version 2.0 (the +//"License"); you may not use this file except in compliance +//with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +//Unless required by applicable law or agreed to in writing, +//software distributed under the License is distributed on an +//"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +//KIND, either express or implied. See the License for the +//specific language governing permissions and limitations +//under the License. +//*/ +//package org.apache.griffin.measure.process +// +//import org.apache.griffin.measure.config.params._ +//import org.apache.griffin.measure.config.params.env._ +//import org.apache.griffin.measure.config.params.user._ +//import org.apache.griffin.measure.config.reader.ParamReaderFactory +//import org.apache.griffin.measure.config.validator.AllParamValidator +//import org.apache.griffin.measure.log.Loggable +//import org.apache.griffin.measure.persist.PersistThreadPool +//import org.apache.griffin.measure.process.engine.DataFrameOprs +//import org.apache.griffin.measure.utils.{HdfsUtil, JsonUtil} +//import org.apache.hadoop.hive.ql.exec.UDF +//import org.apache.spark.{SparkConf, SparkContext} +//import org.apache.spark.sql._ +//import org.apache.spark.sql.catalyst.expressions.GenericRowWithSchema +//import org.apache.spark.sql.expressions.UserDefinedAggregateFunction +//import org.apache.spark.sql.hive.HiveContext +//import org.apache.spark.sql.types._ +//import org.junit.runner.RunWith +//import org.scalatest.junit.JUnitRunner +//import org.scalatest.{BeforeAndAfter, FunSuite, Matchers} +// +//import scala.collection.mutable.WrappedArray +//import scala.util.{Failure, Success, Try} +// +//@RunWith(classOf[JUnitRunner]) +//class JsonParseTest extends FunSuite with Matchers with BeforeAndAfter with Loggable { +// +// var sparkContext: SparkContext = _ +// var sqlContext: SQLContext = _ +// +// before { +// val conf = new SparkConf().setAppName("test json").setMaster("local[*]") +// sparkContext = new SparkContext(conf) +// sparkContext.setLogLevel("WARN") +//// sqlContext = new HiveContext(sparkContext) +// sqlContext = new SQLContext(sparkContext) +// } +// +// test ("json test") { +// // 0. prepare data +//// val dt = +//// """ +//// |{"name": "s1", "age": 12, "items": [1, 2, 3], +//// |"subs": [{"id": 1, "type": "seed"}, {"id": 2, "type": "frog"}], +//// |"inner": {"a": 1, "b": 2}, "jstr": "{\"s1\": \"aaa\", \"s2\": 123}" +//// |}""".stripMargin +//// val rdd0 = sparkContext.parallelize(Seq(dt)).map(Row(_)) +// val rdd0 = sparkContext.textFile("src/test/resources/input.msg").map(Row(_)) +// +// val vtp = StructField("value", StringType) +// val df0 = sqlContext.createDataFrame(rdd0, StructType(Array(vtp))) +// df0.registerTempTable("src") +// +//// val fromJson2Array = (s: String) => { +//// JsonUtil.fromJson[Seq[String]](s) +//// } +//// sqlContext.udf.register("from_json_to_array", fromJson2Array) +//// +//// val df2 = sqlContext.sql("SELECT explode(from_json_to_array(get_json_object(value, '$.seeds'))) as value FROM src") +//// df2.printSchema +//// df2.show(10) +//// df2.registerTempTable("df2") +// // -// val df2 = sqlContext.sql("SELECT explode(from_json_to_array(get_json_object(value, '$.seeds'))) as value FROM src") +// +// // 1. read from json string to extracted json row +//// val readSql = "SELECT value FROM src" +//// val df = sqlContext.sql(readSql) +//// val df = sqlContext.table("src") +//// val rdd = df.map { row => +//// row.getAs[String]("value") +//// } +//// val df1 = sqlContext.read.json(rdd) +//// df1.printSchema +//// df1.show(10) +//// df1.registerTempTable("df1") +// val details = Map[String, Any](("df.name" -> "src")) +// val df1 = DataFrameOprs.fromJson(sqlContext, details) +// df1.registerTempTable("df1") +// +// // 2. extract json array into lines +//// val rdd2 = df1.flatMap { row => +//// row.getAs[WrappedArray[String]]("seeds") +//// } +//// val df2 = sqlContext.read.json(rdd2) +// val df2 = sqlContext.sql("select explode(seeds) as value from df1") +//// val tdf = sqlContext.sql("select name, age, explode(items) as item from df1") +//// tdf.registerTempTable("tdf") +//// val df2 = sqlContext.sql("select struct(name, age, item) as ttt from tdf") // df2.printSchema // df2.show(10) // df2.registerTempTable("df2") - - - - // 1. read from json string to extracted json row -// val readSql = "SELECT value FROM src" +// println(df2.count) +// +// val sql1 = "SELECT value FROM df2" +// val df22 = sqlContext.sql(sql1) +// val rdd22 = df22.map { row => +// row.getAs[String]("value") +// } +// import org.apache.spark.sql.functions._ +// val df23 = sqlContext.read.json(rdd22) +// df23.registerTempTable("df23") +//// df23.withColumn("par", monotonicallyIncreasingId) +// +// val df24 = sqlContext.sql("SELECT url, cast(get_json_object(metadata, '$.tracker.crawlRequestCreateTS') as bigint) as ts FROM df23") +// df24.printSchema +// df24.show(10) +// df24.registerTempTable("df24") +// println(df24.count) +// +//// val df25 = sqlContext.sql("select ") +// +//// +//// // 3. extract json string into row +////// val df3 = sqlContext.sql("select cast(get_json_object(metadata, '$.tracker.crawlRequestCreateTS') as bigint), url from df2") +//// val df3 = sqlContext.sql("select cast(get_json_object(get_json_object(value, '$.metadata'), '$.tracker.crawlRequestCreateTS') as bigint), get_json_object(value, '$.url') from df2") +//// df3.printSchema() +//// df3.show(10) +//// println(df3.count) +// +// +// +//// val df5 = sqlContext.sql("select get_json_object(value, '$.subs') as subs from src") +//// df5.printSchema() +//// df5.show(10) +//// df5.registerTempTable("df5") +//// val rdd5 = df5.map { row => +//// row.getAs[String]("subs") +//// } +//// val df6 = sqlContext.read.json(rdd5) +//// df6.printSchema +//// df6.show(10) +// +// // 2. extract json string to row +//// val df2 = sqlContext.sql("select jstr from df1") +//// val rdd2 = df2.map { row => +//// row.getAs[String]("jstr") +//// } +//// val df22 = sqlContext.read.json(rdd2) +//// df22.printSchema +//// df22.show(100) +//// df22.registerTempTable("df2") +//// +//// val df23 = sqlContext.sql("select json_tuple(jstr, 's1', 's2') from df1") +//// df23.printSchema() +//// df23.show(100) +// +// // 3. extract json array into lines ?? +// +// // 3. flatmap from json row to json row +//// val df3 = sqlContext.sql("select explode(subs) as sub, items from df1") +//// df3.printSchema() +//// df3.show(10) +//// df3.registerTempTable("df3") +//// +//// val df4 = sqlContext.sql("select explode(items) as item, sub from df3") +//// df4.printSchema() +//// df4.show(10) +// +//// sqlContext.udf.register("length", (s: WrappedArray[_]) => s.length) +// // +// // val df2 = sqlContext.sql("SELECT inner from df1") +// // df2.registerTempTable("df2") +// // df2.printSchema +// // df2.show(100) +// +//// def children(colname: String, df: DataFrame): Array[DataFrame] = { +//// val parent = df.schema.fields.filter(_.name == colname).head +//// println(parent) +//// val fields: Array[StructField] = parent.dataType match { +//// case x: StructType => x.fields +//// case _ => Array.empty[StructField] +//// } +//// fields.map(x => col(s"$colname.${x.name}")) +////// fields.foreach(println) +//// } +////// +//// children("inner", df2) +//// +//// df2.select(children("bar", df): _*).printSchema +// +//// val df3 = sqlContext.sql("select inline(subs) from df1") +//// df3.printSchema() +//// df3.show(100) +// +//// val rdd2 = df2.flatMap { row => +//// row.getAs[GenericRowWithSchema]("inner") :: Nil +//// } +//// +//// rdd2. +// +//// val funcs = sqlContext.sql("show functions") +//// funcs.printSchema() +//// funcs.show(1000) +//// +//// val desc = sqlContext.sql("describe function inline") +//// desc.printSchema() +//// desc.show(100) +// +// // +// +// } +// +// test ("json test 2") { +// val rdd0 = sparkContext.textFile("src/test/resources/output.msg").map(Row(_)) +// +// val vtp = StructField("value", StringType) +// val df0 = sqlContext.createDataFrame(rdd0, StructType(Array(vtp))) +// df0.registerTempTable("tgt") +// +//// val fromJson2StringArray = (s: String) => { +//// val seq = JsonUtil.fromJson[Seq[Any]](s) +//// seq.map(i => JsonUtil.toJson(i)) +//// } +//// sqlContext.udf.register("from_json_to_string_array", fromJson2StringArray) +//// +//// val df2 = sqlContext.sql("SELECT from_json_to_string_array(get_json_object(value, '$.groups[0].attrsList')) as value FROM tgt") +//// df2.printSchema() +//// df2.show(10) +//// df2.registerTempTable("df2") +//// +//// val indexOfStringArray = (sa: String, ) +// +// +// // 1. read from json string to extracted json row +// val readSql = "SELECT value FROM tgt" // val df = sqlContext.sql(readSql) -// val df = sqlContext.table("src") // val rdd = df.map { row => // row.getAs[String]("value") // } @@ -93,346 +251,93 @@ class JsonParseTest extends FunSuite with Matchers with BeforeAndAfter with Logg // df1.printSchema // df1.show(10) // df1.registerTempTable("df1") - val details = Map[String, Any](("df.name" -> "src")) - val df1 = DataFrameOprs.fromJson(sqlContext, details) - df1.registerTempTable("df1") - - // 2. extract json array into lines -// val rdd2 = df1.flatMap { row => -// row.getAs[WrappedArray[String]]("seeds") -// } -// val df2 = sqlContext.read.json(rdd2) - val df2 = sqlContext.sql("select explode(seeds) as value from df1") -// val tdf = sqlContext.sql("select name, age, explode(items) as item from df1") -// tdf.registerTempTable("tdf") -// val df2 = sqlContext.sql("select struct(name, age, item) as ttt from tdf") - df2.printSchema - df2.show(10) - df2.registerTempTable("df2") - println(df2.count) - - val sql1 = "SELECT value FROM df2" - val df22 = sqlContext.sql(sql1) - val rdd22 = df22.map { row => - row.getAs[String]("value") - } - import org.apache.spark.sql.functions._ - val df23 = sqlContext.read.json(rdd22) - df23.registerTempTable("df23") -// df23.withColumn("par", monotonicallyIncreasingId) - - val df24 = sqlContext.sql("SELECT url, cast(get_json_object(metadata, '$.tracker.crawlRequestCreateTS') as bigint) as ts FROM df23") - df24.printSchema - df24.show(10) - df24.registerTempTable("df24") - println(df24.count) - -// val df25 = sqlContext.sql("select ") - -// -// // 3. extract json string into row -//// val df3 = sqlContext.sql("select cast(get_json_object(metadata, '$.tracker.crawlRequestCreateTS') as bigint), url from df2") -// val df3 = sqlContext.sql("select cast(get_json_object(get_json_object(value, '$.metadata'), '$.tracker.crawlRequestCreateTS') as bigint), get_json_object(value, '$.url') from df2") -// df3.printSchema() -// df3.show(10) -// println(df3.count) - - - -// val df5 = sqlContext.sql("select get_json_object(value, '$.subs') as subs from src") -// df5.printSchema() -// df5.show(10) -// df5.registerTempTable("df5") -// val rdd5 = df5.map { row => -// row.getAs[String]("subs") -// } -// val df6 = sqlContext.read.json(rdd5) -// df6.printSchema -// df6.show(10) - - // 2. extract json string to row -// val df2 = sqlContext.sql("select jstr from df1") -// val rdd2 = df2.map { row => -// row.getAs[String]("jstr") +// +// +// val df2 = sqlContext.sql("select groups[0].attrsList as attrs from df1") +// df2.printSchema +// df2.show(10) +// df2.registerTempTable("df2") +// println(df2.count) +// +// val indexOf = (arr: Seq[String], v: String) => { +// arr.indexOf(v) // } -// val df22 = sqlContext.read.json(rdd2) -// df22.printSchema -// df22.show(100) -// df22.registerTempTable("df2") -// -// val df23 = sqlContext.sql("select json_tuple(jstr, 's1', 's2') from df1") -// df23.printSchema() -// df23.show(100) - - // 3. extract json array into lines ?? - - // 3. flatmap from json row to json row -// val df3 = sqlContext.sql("select explode(subs) as sub, items from df1") +// sqlContext.udf.register("index_of", indexOf) +// +// val df3 = sqlContext.sql("select attrs.values[index_of(attrs.name, 'URL')][0] as url, cast(get_json_object(attrs.values[index_of(attrs.name, 'CRAWLMETADATA')][0], '$.tracker.crawlRequestCreateTS') as bigint) as ts from df2") // df3.printSchema() // df3.show(10) // df3.registerTempTable("df3") +// } // -// val df4 = sqlContext.sql("select explode(items) as item, sub from df3") -// df4.printSchema() -// df4.show(10) - -// sqlContext.udf.register("length", (s: WrappedArray[_]) => s.length) - // - // val df2 = sqlContext.sql("SELECT inner from df1") - // df2.registerTempTable("df2") - // df2.printSchema - // df2.show(100) - -// def children(colname: String, df: DataFrame): Array[DataFrame] = { -// val parent = df.schema.fields.filter(_.name == colname).head -// println(parent) -// val fields: Array[StructField] = parent.dataType match { -// case x: StructType => x.fields -// case _ => Array.empty[StructField] -// } -// fields.map(x => col(s"$colname.${x.name}")) -//// fields.foreach(println) -// } -//// -// children("inner", df2) +// test ("testing") { +// val dt = +// """ +// |{"name": "age", "age": 12, "items": [1, 2, 3], +// |"subs": [{"id": 1, "type": "seed"}, {"id": 2, "type": "frog"}], +// |"inner": {"a": 1, "b": 2}, "jstr": "{\"s1\": \"aaa\", \"s2\": 123}", "b": true +// |}""".stripMargin +// val rdd = sparkContext.parallelize(Seq(dt)).map(Row(_)) +// val vtp = StructField("value", StringType) +// val df = sqlContext.createDataFrame(rdd, StructType(Array(vtp))) +// df.registerTempTable("df") // -// df2.select(children("bar", df): _*).printSchema - -// val df3 = sqlContext.sql("select inline(subs) from df1") -// df3.printSchema() -// df3.show(100) - -// val rdd2 = df2.flatMap { row => -// row.getAs[GenericRowWithSchema]("inner") :: Nil -// } +// val df1 = sqlContext.read.json(sqlContext.sql("select * from df").map(r => r.getAs[String]("value"))) +// df1.printSchema() +// df1.show(10) +// df1.registerTempTable("df1") // -// rdd2. - -// val funcs = sqlContext.sql("show functions") -// funcs.printSchema() -// funcs.show(1000) -// -// val desc = sqlContext.sql("describe function inline") -// desc.printSchema() -// desc.show(100) - - // - - } - - test ("json test 2") { - val rdd0 = sparkContext.textFile("src/test/resources/output.msg").map(Row(_)) - - val vtp = StructField("value", StringType) - val df0 = sqlContext.createDataFrame(rdd0, StructType(Array(vtp))) - df0.registerTempTable("tgt") - -// val fromJson2StringArray = (s: String) => { -// val seq = JsonUtil.fromJson[Seq[Any]](s) -// seq.map(i => JsonUtil.toJson(i)) +// val test = (s: String) => { +// s.toInt // } -// sqlContext.udf.register("from_json_to_string_array", fromJson2StringArray) +// sqlContext.udf.register("to_int", test) // -// val df2 = sqlContext.sql("SELECT from_json_to_string_array(get_json_object(value, '$.groups[0].attrsList')) as value FROM tgt") +// val df2 = sqlContext.sql("select (b) as aa, inner.a from df1 where age = to_int(\"12\")") // df2.printSchema() // df2.show(10) +// } +// +// test ("test input only sql") { +// val rdd0 = sparkContext.textFile("src/test/resources/input.msg").map(Row(_)) +// +// val vtp = StructField("value", StringType) +// val df0 = sqlContext.createDataFrame(rdd0, StructType(Array(vtp))) +// df0.registerTempTable("src") +// df0.show(10) +// +// // 1. read from json string to extracted json row +// val df1 = sqlContext.sql("SELECT get_json_object(value, '$.seeds') as seeds FROM src") +// df1.printSchema +// df1.show(10) +// df1.registerTempTable("df1") +// +// val json2StringArray: (String) => Seq[String] = (s: String) => { +// val seq = JsonUtil.fromJson[Seq[String]](s) +//// seq.map(i => JsonUtil.toJson(i)) +// seq +// } +// sqlContext.udf.register("json_to_string_array", json2StringArray) +// +// val df2 = sqlContext.sql("SELECT explode(json_to_string_array(seeds)) as seed FROM df1") +// df2.printSchema +// df2.show(10) // df2.registerTempTable("df2") // -// val indexOfStringArray = (sa: String, ) - - - // 1. read from json string to extracted json row - val readSql = "SELECT value FROM tgt" - val df = sqlContext.sql(readSql) - val rdd = df.map { row => - row.getAs[String]("value") - } - val df1 = sqlContext.read.json(rdd) - df1.printSchema - df1.show(10) - df1.registerTempTable("df1") - - - val df2 = sqlContext.sql("select groups[0].attrsList as attrs from df1") - df2.printSchema - df2.show(10) - df2.registerTempTable("df2") - println(df2.count) - - val indexOf = (arr: Seq[String], v: String) => { - arr.indexOf(v) - } - sqlContext.udf.register("index_of", indexOf) - - val df3 = sqlContext.sql("select attrs.values[index_of(attrs.name, 'URL')][0] as url, cast(get_json_object(attrs.values[index_of(attrs.name, 'CRAWLMETADATA')][0], '$.tracker.crawlRequestCreateTS') as bigint) as ts from df2") - df3.printSchema() - df3.show(10) - df3.registerTempTable("df3") - } - - test ("testing") { - val dt = - """ - |{"name": "age", "age": 12, "items": [1, 2, 3], - |"subs": [{"id": 1, "type": "seed"}, {"id": 2, "type": "frog"}], - |"inner": {"a": 1, "b": 2}, "jstr": "{\"s1\": \"aaa\", \"s2\": 123}", "b": true - |}""".stripMargin - val rdd = sparkContext.parallelize(Seq(dt)).map(Row(_)) - val vtp = StructField("value", StringType) - val df = sqlContext.createDataFrame(rdd, StructType(Array(vtp))) - df.registerTempTable("df") - - val df1 = sqlContext.read.json(sqlContext.sql("select * from df").map(r => r.getAs[String]("value"))) - df1.printSchema() - df1.show(10) - df1.registerTempTable("df1") - - val test = (s: String) => { - s.toInt - } - sqlContext.udf.register("to_int", test) - - val df2 = sqlContext.sql("select (b) as aa, inner.a from df1 where age = to_int(\"12\")") - df2.printSchema() - df2.show(10) - } - - test ("test input only sql") { - val rdd0 = sparkContext.textFile("src/test/resources/input.msg").map(Row(_)) - - val vtp = StructField("value", StringType) - val df0 = sqlContext.createDataFrame(rdd0, StructType(Array(vtp))) - df0.registerTempTable("src") - df0.show(10) - - // 1. read from json string to extracted json row - val df1 = sqlContext.sql("SELECT get_json_object(value, '$.seeds') as seeds FROM src") - df1.printSchema - df1.show(10) - df1.registerTempTable("df1") - - val json2StringArray: (String) => Seq[String] = (s: String) => { - val seq = JsonUtil.fromJson[Seq[String]](s) -// seq.map(i => JsonUtil.toJson(i)) - seq - } - sqlContext.udf.register("json_to_string_array", json2StringArray) - - val df2 = sqlContext.sql("SELECT explode(json_to_string_array(seeds)) as seed FROM df1") - df2.printSchema - df2.show(10) - df2.registerTempTable("df2") - - - val df3 = sqlContext.sql("SELECT get_json_object(seed, '$.url') as url, cast(get_json_object(get_json_object(seed, '$.metadata'), '$.tracker.crawlRequestCreateTS') as bigint) as ts FROM df2") - df3.printSchema - df3.show(10) - } - - test ("test output only sql") { - val rdd0 = sparkContext.textFile("src/test/resources/output.msg").map(Row(_)) - - val vtp = StructField("value", StringType) - val df0 = sqlContext.createDataFrame(rdd0, StructType(Array(vtp))) - df0.registerTempTable("tgt") - df0.printSchema() - df0.show(10) - - val json2StringArray: (String) => Seq[String] = (s: String) => { - JsonUtil.fromJson[Seq[String]](s) - } - sqlContext.udf.register("json_to_string_array", json2StringArray) - - val json2StringJsonArray: (String) => Seq[String] = (s: String) => { - val seq = JsonUtil.fromJson[Seq[Any]](s) - seq.map(i => JsonUtil.toJson(i)) - } - sqlContext.udf.register("json_to_string_json_array", json2StringJsonArray) - - val indexOf = (arr: Seq[String], v: String) => { - arr.indexOf(v) - } - sqlContext.udf.register("index_of", indexOf) - - val indexOfField = (arr: Seq[String], k: String, v: String) => { - val seq = arr.flatMap { item => - JsonUtil.fromJson[Map[String, Any]](item).get(k) - } - seq.indexOf(v) - } - sqlContext.udf.register("index_of_field", indexOfField) - - // 1. read from json string to extracted json row - val df1 = sqlContext.sql("SELECT get_json_object(value, '$.groups[0].attrsList') as attrs FROM tgt") - df1.printSchema - df1.show(10) - df1.registerTempTable("df1") - - val df2 = sqlContext.sql("SELECT json_to_string_json_array(attrs) as attrs FROM df1") - df2.printSchema() - df2.show(10) - df2.registerTempTable("df2") - - val df3 = sqlContext.sql("SELECT attrs[index_of_field(attrs, 'name', 'URL')] as attr1, attrs[index_of_field(attrs, 'name', 'CRAWLMETADATA')] as attr2 FROM df2") - df3.printSchema() - df3.show(10) - df3.registerTempTable("df3") - - val df4 = sqlContext.sql("SELECT json_to_string_array(get_json_object(attr1, '$.values'))[0], cast(get_json_object(json_to_string_array(get_json_object(attr2, '$.values'))[0], '$.tracker.crawlRequestCreateTS') as bigint) FROM df3") - df4.printSchema() - df4.show(10) - } - - test ("test from json") { - val fromJson2Map = (str: String) => { - val a = JsonUtil.fromJson[Map[String, Any]](str) - a.mapValues { v => - v match { - case t: String => t - case _ => JsonUtil.toJson(v) - } - } - } - sqlContext.udf.register("from_json_to_map", fromJson2Map) - - val fromJson2Array = (str: String) => { - val a = JsonUtil.fromJson[Seq[Any]](str) - a.map { v => - v match { - case t: String => t - case _ => JsonUtil.toJson(v) - } - } - } - sqlContext.udf.register("from_json_to_array", fromJson2Array) - - // ======================== - - val srdd = sparkContext.textFile("src/test/resources/input.msg").map(Row(_)) - val svtp = StructField("value", StringType) - val sdf0 = sqlContext.createDataFrame(srdd, StructType(Array(svtp))) - sdf0.registerTempTable("sdf0") - sdf0.show(10) - - // 1. read from json string to extracted json row - val sdf1 = sqlContext.sql("SELECT explode(from_json_to_array(get_json_object(value, '$.seeds'))) as seed FROM sdf0") - sdf1.printSchema - sdf1.show(10) - sdf1.registerTempTable("sdf1") - - val sdf2 = sqlContext.sql("SELECT get_json_object(seed, '$.url') as url, cast(get_json_object(get_json_object(seed, '$.metadata'), '$.tracker.crawlRequestCreateTS') as bigint) as ts FROM sdf1") - sdf2.printSchema - sdf2.show(10) - - // --------------------------------------- - - val trdd = sparkContext.textFile("src/test/resources/output.msg").map(Row(_)) - val tvtp = StructField("value", StringType) - val tdf0 = sqlContext.createDataFrame(trdd, StructType(Array(tvtp))) - tdf0.registerTempTable("tdf0") - tdf0.printSchema() - tdf0.show(10) - +// +// val df3 = sqlContext.sql("SELECT get_json_object(seed, '$.url') as url, cast(get_json_object(get_json_object(seed, '$.metadata'), '$.tracker.crawlRequestCreateTS') as bigint) as ts FROM df2") +// df3.printSchema +// df3.show(10) +// } +// +// test ("test output only sql") { +// val rdd0 = sparkContext.textFile("src/test/resources/output.msg").map(Row(_)) +// +// val vtp = StructField("value", StringType) +// val df0 = sqlContext.createDataFrame(rdd0, StructType(Array(vtp))) +// df0.registerTempTable("tgt") +// df0.printSchema() +// df0.show(10) +// // val json2StringArray: (String) => Seq[String] = (s: String) => { // JsonUtil.fromJson[Seq[String]](s) // } @@ -449,83 +354,178 @@ class JsonParseTest extends FunSuite with Matchers with BeforeAndAfter with Logg // } // sqlContext.udf.register("index_of", indexOf) // - val indexOfField = (arr: Seq[String], k: String, v: String) => { - val seq = arr.flatMap { item => - JsonUtil.fromJson[Map[String, Any]](item).get(k) - } - seq.indexOf(v) - } - sqlContext.udf.register("index_of_field", indexOfField) - - // 1. read from json string to extracted json row -// val df1 = sqlContext.sql("SELECT get_json_object(value, '$.groups[0].attrsList') as attrs FROM tdf0") - val tdf1 = sqlContext.sql("SELECT from_json_to_array(get_json_object(value, '$.groups[0].attrsList')) as attrs FROM tdf0") - tdf1.printSchema - tdf1.show(10) - tdf1.registerTempTable("tdf1") - -// val tdf2 = sqlContext.sql("SELECT attrs[index_of_field(attrs, 'name', 'URL')] as attr1, attrs[index_of_field(attrs, 'name', 'CRAWLMETADATA')] as attr2 FROM tdf1") -// tdf2.printSchema() -// tdf2.show(10) -// tdf2.registerTempTable("tdf2") - - val tdf3 = sqlContext.sql("SELECT from_json_to_array(get_json_object(attrs[index_of_field(attrs, 'name', 'URL')], '$.values'))[0] as url, cast(get_json_object(from_json_to_array(get_json_object(attrs[index_of_field(attrs, 'name', 'CRAWLMETADATA')], '$.values'))[0], '$.tracker.crawlRequestCreateTS') as bigint) as ts FROM tdf1") - tdf3.printSchema() - tdf3.show(10) - } - - test ("sql functions") { - val functions = sqlContext.sql("show functions") - functions.printSchema() - functions.show(10) - - val functionNames = functions.map(_.getString(0)).collect - functionNames.foreach(println) - } - - test ("test text file read") { - val partitionPaths = Seq[String]( - "hdfs://localhost/griffin/streaming/dump/source/418010/25080625/1504837518000", - "hdfs://localhost/griffin/streaming/dump/target/418010/25080625/1504837518000") - val df = sqlContext.read.json(partitionPaths: _*) - df.printSchema() - df.show(10) - } - - test ("list paths") { - val filePath = "hdfs://localhost/griffin/streaming/dump/source" - val partitionRanges = List[(Long, Long)]((0, 0), (-2, 0)) - val partitionPaths = listPathsBetweenRanges(filePath :: Nil, partitionRanges) - println(partitionPaths) - } - - private def listPathsBetweenRanges(paths: List[String], - partitionRanges: List[(Long, Long)] - ): List[String] = { - partitionRanges match { - case Nil => paths - case head :: tail => { - val (lb, ub) = head - val curPaths = paths.flatMap { path => - val names = HdfsUtil.listSubPathsByType(path, "dir").toList - println(names) - names.filter { name => - str2Long(name) match { - case Some(t) => (t >= lb) && (t <= ub) - case _ => false - } - }.map(HdfsUtil.getHdfsFilePath(path, _)) - } - listPathsBetweenRanges(curPaths, tail) - } - } - } - - private def str2Long(str: String): Option[Long] = { - try { - Some(str.toLong) - } catch { - case e: Throwable => None - } - } -} \ No newline at end of file +// val indexOfField = (arr: Seq[String], k: String, v: String) => { +// val seq = arr.flatMap { item => +// JsonUtil.fromJson[Map[String, Any]](item).get(k) +// } +// seq.indexOf(v) +// } +// sqlContext.udf.register("index_of_field", indexOfField) +// +// // 1. read from json string to extracted json row +// val df1 = sqlContext.sql("SELECT get_json_object(value, '$.groups[0].attrsList') as attrs FROM tgt") +// df1.printSchema +// df1.show(10) +// df1.registerTempTable("df1") +// +// val df2 = sqlContext.sql("SELECT json_to_string_json_array(attrs) as attrs FROM df1") +// df2.printSchema() +// df2.show(10) +// df2.registerTempTable("df2") +// +// val df3 = sqlContext.sql("SELECT attrs[index_of_field(attrs, 'name', 'URL')] as attr1, attrs[index_of_field(attrs, 'name', 'CRAWLMETADATA')] as attr2 FROM df2") +// df3.printSchema() +// df3.show(10) +// df3.registerTempTable("df3") +// +// val df4 = sqlContext.sql("SELECT json_to_string_array(get_json_object(attr1, '$.values'))[0], cast(get_json_object(json_to_string_array(get_json_object(attr2, '$.values'))[0], '$.tracker.crawlRequestCreateTS') as bigint) FROM df3") +// df4.printSchema() +// df4.show(10) +// } +// +// test ("test from json") { +// val fromJson2Map = (str: String) => { +// val a = JsonUtil.fromJson[Map[String, Any]](str) +// a.mapValues { v => +// v match { +// case t: String => t +// case _ => JsonUtil.toJson(v) +// } +// } +// } +// sqlContext.udf.register("from_json_to_map", fromJson2Map) +// +// val fromJson2Array = (str: String) => { +// val a = JsonUtil.fromJson[Seq[Any]](str) +// a.map { v => +// v match { +// case t: String => t +// case _ => JsonUtil.toJson(v) +// } +// } +// } +// sqlContext.udf.register("from_json_to_array", fromJson2Array) +// +// // ======================== +// +// val srdd = sparkContext.textFile("src/test/resources/input.msg").map(Row(_)) +// val svtp = StructField("value", StringType) +// val sdf0 = sqlContext.createDataFrame(srdd, StructType(Array(svtp))) +// sdf0.registerTempTable("sdf0") +// sdf0.show(10) +// +// // 1. read from json string to extracted json row +// val sdf1 = sqlContext.sql("SELECT explode(from_json_to_array(get_json_object(value, '$.seeds'))) as seed FROM sdf0") +// sdf1.printSchema +// sdf1.show(10) +// sdf1.registerTempTable("sdf1") +// +// val sdf2 = sqlContext.sql("SELECT get_json_object(seed, '$.url') as url, cast(get_json_object(get_json_object(seed, '$.metadata'), '$.tracker.crawlRequestCreateTS') as bigint) as ts FROM sdf1") +// sdf2.printSchema +// sdf2.show(10) +// +// // --------------------------------------- +// +// val trdd = sparkContext.textFile("src/test/resources/output.msg").map(Row(_)) +// val tvtp = StructField("value", StringType) +// val tdf0 = sqlContext.createDataFrame(trdd, StructType(Array(tvtp))) +// tdf0.registerTempTable("tdf0") +// tdf0.printSchema() +// tdf0.show(10) +// +//// val json2StringArray: (String) => Seq[String] = (s: String) => { +//// JsonUtil.fromJson[Seq[String]](s) +//// } +//// sqlContext.udf.register("json_to_string_array", json2StringArray) +//// +//// val json2StringJsonArray: (String) => Seq[String] = (s: String) => { +//// val seq = JsonUtil.fromJson[Seq[Any]](s) +//// seq.map(i => JsonUtil.toJson(i)) +//// } +//// sqlContext.udf.register("json_to_string_json_array", json2StringJsonArray) +//// +//// val indexOf = (arr: Seq[String], v: String) => { +//// arr.indexOf(v) +//// } +//// sqlContext.udf.register("index_of", indexOf) +//// +// val indexOfField = (arr: Seq[String], k: String, v: String) => { +// val seq = arr.flatMap { item => +// JsonUtil.fromJson[Map[String, Any]](item).get(k) +// } +// seq.indexOf(v) +// } +// sqlContext.udf.register("index_of_field", indexOfField) +// +// // 1. read from json string to extracted json row +//// val df1 = sqlContext.sql("SELECT get_json_object(value, '$.groups[0].attrsList') as attrs FROM tdf0") +// val tdf1 = sqlContext.sql("SELECT from_json_to_array(get_json_object(value, '$.groups[0].attrsList')) as attrs FROM tdf0") +// tdf1.printSchema +// tdf1.show(10) +// tdf1.registerTempTable("tdf1") +// +//// val tdf2 = sqlContext.sql("SELECT attrs[index_of_field(attrs, 'name', 'URL')] as attr1, attrs[index_of_field(attrs, 'name', 'CRAWLMETADATA')] as attr2 FROM tdf1") +//// tdf2.printSchema() +//// tdf2.show(10) +//// tdf2.registerTempTable("tdf2") +// +// val tdf3 = sqlContext.sql("SELECT from_json_to_array(get_json_object(attrs[index_of_field(attrs, 'name', 'URL')], '$.values'))[0] as url, cast(get_json_object(from_json_to_array(get_json_object(attrs[index_of_field(attrs, 'name', 'CRAWLMETADATA')], '$.values'))[0], '$.tracker.crawlRequestCreateTS') as bigint) as ts FROM tdf1") +// tdf3.printSchema() +// tdf3.show(10) +// } +// +// test ("sql functions") { +// val functions = sqlContext.sql("show functions") +// functions.printSchema() +// functions.show(10) +// +// val functionNames = functions.map(_.getString(0)).collect +// functionNames.foreach(println) +// } +// +// test ("test text file read") { +// val partitionPaths = Seq[String]( +// "hdfs://localhost/griffin/streaming/dump/source/418010/25080625/1504837518000", +// "hdfs://localhost/griffin/streaming/dump/target/418010/25080625/1504837518000") +// val df = sqlContext.read.json(partitionPaths: _*) +// df.printSchema() +// df.show(10) +// } +// +// test ("list paths") { +// val filePath = "hdfs://localhost/griffin/streaming/dump/source" +// val partitionRanges = List[(Long, Long)]((0, 0), (-2, 0)) +// val partitionPaths = listPathsBetweenRanges(filePath :: Nil, partitionRanges) +// println(partitionPaths) +// } +// +// private def listPathsBetweenRanges(paths: List[String], +// partitionRanges: List[(Long, Long)] +// ): List[String] = { +// partitionRanges match { +// case Nil => paths +// case head :: tail => { +// val (lb, ub) = head +// val curPaths = paths.flatMap { path => +// val names = HdfsUtil.listSubPathsByType(path, "dir").toList +// println(names) +// names.filter { name => +// str2Long(name) match { +// case Some(t) => (t >= lb) && (t <= ub) +// case _ => false +// } +// }.map(HdfsUtil.getHdfsFilePath(path, _)) +// } +// listPathsBetweenRanges(curPaths, tail) +// } +// } +// } +// +// private def str2Long(str: String): Option[Long] = { +// try { +// Some(str.toLong) +// } catch { +// case e: Throwable => None +// } +// } +//} \ No newline at end of file diff --git a/measure/src/test/scala/org/apache/griffin/measure/process/StreamingProcessTest.scala b/measure/src/test/scala/org/apache/griffin/measure/process/StreamingProcessTest.scala index 07b7c5e08..caecc9cbe 100644 --- a/measure/src/test/scala/org/apache/griffin/measure/process/StreamingProcessTest.scala +++ b/measure/src/test/scala/org/apache/griffin/measure/process/StreamingProcessTest.scala @@ -1,147 +1,147 @@ -/* -Licensed to the Apache Software Foundation (ASF) under one -or more contributor license agreements. See the NOTICE file -distributed with this work for additional information -regarding copyright ownership. The ASF licenses this file -to you under the Apache License, Version 2.0 (the -"License"); you may not use this file except in compliance -with the License. You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, -software distributed under the License is distributed on an -"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -KIND, either express or implied. See the License for the -specific language governing permissions and limitations -under the License. -*/ -package org.apache.griffin.measure.process - -import org.apache.griffin.measure.config.params._ -import org.apache.griffin.measure.config.params.env._ -import org.apache.griffin.measure.config.params.user._ -import org.apache.griffin.measure.config.reader.ParamReaderFactory -import org.apache.griffin.measure.config.validator.AllParamValidator -import org.apache.griffin.measure.log.Loggable -import org.apache.griffin.measure.persist.PersistThreadPool -import org.junit.runner.RunWith -import org.scalatest.junit.JUnitRunner -import org.scalatest.{BeforeAndAfter, FunSuite, Matchers} - -import scala.util.{Failure, Success, Try} - -@RunWith(classOf[JUnitRunner]) -class StreamingProcessTest extends FunSuite with Matchers with BeforeAndAfter with Loggable { - - val envFile = "src/test/resources/env-streaming.json" -// val confFile = "src/test/resources/config-test-accuracy-streaming-multids.json" - val confFile = "src/test/resources/config-test-accuracy-streaming.json" -// val confFile = "src/test/resources/config-test-profiling-streaming.json" - - val envFsType = "local" - val userFsType = "local" - - val args = Array(envFile, confFile) - - var allParam: AllParam = _ - - before { - // read param files - val envParam = readParamFile[EnvParam](envFile, envFsType) match { - case Success(p) => p - case Failure(ex) => { - error(ex.getMessage) - sys.exit(-2) - } - } - val userParam = readParamFile[UserParam](confFile, userFsType) match { - case Success(p) => p - case Failure(ex) => { - error(ex.getMessage) - sys.exit(-2) - } - } - allParam = AllParam(envParam, userParam) - - // validate param files - validateParams(allParam) match { - case Failure(ex) => { - error(ex.getMessage) - sys.exit(-3) - } - case _ => { - info("params validation pass") - } - } - } - - test ("streaming process") { - val procType = ProcessType(allParam.userParam.procType) - val proc: DqProcess = procType match { - case BatchProcessType => BatchDqProcess(allParam) - case StreamingProcessType => StreamingDqProcess(allParam) - case _ => { - error(s"${procType} is unsupported process type!") - sys.exit(-4) - } - } - - // process init - proc.init match { - case Success(_) => { - info("process init success") - } - case Failure(ex) => { - error(s"process init error: ${ex.getMessage}") - shutdown - sys.exit(-5) - } - } - - // process run - proc.run match { - case Success(_) => { - info("process run success") - } - case Failure(ex) => { - error(s"process run error: ${ex.getMessage}") - - if (proc.retriable) { - throw ex - } else { - shutdown - sys.exit(-5) - } - } - } - - // process end - proc.end match { - case Success(_) => { - info("process end success") - } - case Failure(ex) => { - error(s"process end error: ${ex.getMessage}") - shutdown - sys.exit(-5) - } - } - - shutdown - } - - private def readParamFile[T <: Param](file: String, fsType: String)(implicit m : Manifest[T]): Try[T] = { - val paramReader = ParamReaderFactory.getParamReader(file, fsType) - paramReader.readConfig[T] - } - - private def validateParams(allParam: AllParam): Try[Boolean] = { - val allParamValidator = AllParamValidator() - allParamValidator.validate(allParam) - } - - private def shutdown(): Unit = { - PersistThreadPool.shutdown - } -} +///* +//Licensed to the Apache Software Foundation (ASF) under one +//or more contributor license agreements. See the NOTICE file +//distributed with this work for additional information +//regarding copyright ownership. The ASF licenses this file +//to you under the Apache License, Version 2.0 (the +//"License"); you may not use this file except in compliance +//with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +//Unless required by applicable law or agreed to in writing, +//software distributed under the License is distributed on an +//"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +//KIND, either express or implied. See the License for the +//specific language governing permissions and limitations +//under the License. +//*/ +//package org.apache.griffin.measure.process +// +//import org.apache.griffin.measure.config.params._ +//import org.apache.griffin.measure.config.params.env._ +//import org.apache.griffin.measure.config.params.user._ +//import org.apache.griffin.measure.config.reader.ParamReaderFactory +//import org.apache.griffin.measure.config.validator.AllParamValidator +//import org.apache.griffin.measure.log.Loggable +//import org.apache.griffin.measure.persist.PersistThreadPool +//import org.junit.runner.RunWith +//import org.scalatest.junit.JUnitRunner +//import org.scalatest.{BeforeAndAfter, FunSuite, Matchers} +// +//import scala.util.{Failure, Success, Try} +// +//@RunWith(classOf[JUnitRunner]) +//class StreamingProcessTest extends FunSuite with Matchers with BeforeAndAfter with Loggable { +// +// val envFile = "src/test/resources/env-streaming.json" +//// val confFile = "src/test/resources/config-test-accuracy-streaming-multids.json" +// val confFile = "src/test/resources/config-test-accuracy-streaming.json" +//// val confFile = "src/test/resources/config-test-profiling-streaming.json" +// +// val envFsType = "local" +// val userFsType = "local" +// +// val args = Array(envFile, confFile) +// +// var allParam: AllParam = _ +// +// before { +// // read param files +// val envParam = readParamFile[EnvParam](envFile, envFsType) match { +// case Success(p) => p +// case Failure(ex) => { +// error(ex.getMessage) +// sys.exit(-2) +// } +// } +// val userParam = readParamFile[UserParam](confFile, userFsType) match { +// case Success(p) => p +// case Failure(ex) => { +// error(ex.getMessage) +// sys.exit(-2) +// } +// } +// allParam = AllParam(envParam, userParam) +// +// // validate param files +// validateParams(allParam) match { +// case Failure(ex) => { +// error(ex.getMessage) +// sys.exit(-3) +// } +// case _ => { +// info("params validation pass") +// } +// } +// } +// +// test ("streaming process") { +// val procType = ProcessType(allParam.userParam.procType) +// val proc: DqProcess = procType match { +// case BatchProcessType => BatchDqProcess(allParam) +// case StreamingProcessType => StreamingDqProcess(allParam) +// case _ => { +// error(s"${procType} is unsupported process type!") +// sys.exit(-4) +// } +// } +// +// // process init +// proc.init match { +// case Success(_) => { +// info("process init success") +// } +// case Failure(ex) => { +// error(s"process init error: ${ex.getMessage}") +// shutdown +// sys.exit(-5) +// } +// } +// +// // process run +// proc.run match { +// case Success(_) => { +// info("process run success") +// } +// case Failure(ex) => { +// error(s"process run error: ${ex.getMessage}") +// +// if (proc.retriable) { +// throw ex +// } else { +// shutdown +// sys.exit(-5) +// } +// } +// } +// +// // process end +// proc.end match { +// case Success(_) => { +// info("process end success") +// } +// case Failure(ex) => { +// error(s"process end error: ${ex.getMessage}") +// shutdown +// sys.exit(-5) +// } +// } +// +// shutdown +// } +// +// private def readParamFile[T <: Param](file: String, fsType: String)(implicit m : Manifest[T]): Try[T] = { +// val paramReader = ParamReaderFactory.getParamReader(file, fsType) +// paramReader.readConfig[T] +// } +// +// private def validateParams(allParam: AllParam): Try[Boolean] = { +// val allParamValidator = AllParamValidator() +// allParamValidator.validate(allParam) +// } +// +// private def shutdown(): Unit = { +// PersistThreadPool.shutdown +// } +//} diff --git a/measure/src/test/scala/org/apache/griffin/measure/rule/dsl/parser/BasicParserTest.scala b/measure/src/test/scala/org/apache/griffin/measure/rule/dsl/parser/BasicParserTest.scala index d8c95313d..823554d3f 100644 --- a/measure/src/test/scala/org/apache/griffin/measure/rule/dsl/parser/BasicParserTest.scala +++ b/measure/src/test/scala/org/apache/griffin/measure/rule/dsl/parser/BasicParserTest.scala @@ -202,4 +202,18 @@ class BasicParserTest extends FunSuite with Matchers with BeforeAndAfter { println(result.get.desc) } + test ("select clause") { + val rule = "select source.user_id, item, source.func()" + val result = parser.parseAll(parser.selectClause, rule) + result.successful should be (true) + println(result.get.desc) + } + + test ("from clause") { + val rule = "from source" + val result = parser.parseAll(parser.fromClause, rule) + result.successful should be (true) + println(result.get.desc) + } + } From 415259623f10f8013e74e25b82802a71efc44716 Mon Sep 17 00:00:00 2001 From: Lionel Liu Date: Tue, 10 Oct 2017 14:23:38 +0800 Subject: [PATCH 097/111] ignore derby.log --- .gitignore | 2 ++ 1 file changed, 2 insertions(+) diff --git a/.gitignore b/.gitignore index 3764dc947..8ba841b9d 100644 --- a/.gitignore +++ b/.gitignore @@ -31,3 +31,5 @@ ui/bower_components/* ui/node_modules/* ui/debug.log ui/package-lock.json + +derby.log From 57190998963007122fdabd1c57d9a12a5a547718 Mon Sep 17 00:00:00 2001 From: Lionel Liu Date: Tue, 10 Oct 2017 14:39:49 +0800 Subject: [PATCH 098/111] remove spark test --- measure/src/test/resources/input.msg | 1 - measure/src/test/resources/output.msg | 1 - .../griffin/measure/utils/HdfsUtilTest.scala | 104 +++++++++--------- 3 files changed, 52 insertions(+), 54 deletions(-) delete mode 100644 measure/src/test/resources/input.msg delete mode 100644 measure/src/test/resources/output.msg diff --git a/measure/src/test/resources/input.msg b/measure/src/test/resources/input.msg deleted file mode 100644 index edb4619e0..000000000 --- a/measure/src/test/resources/input.msg +++ /dev/null @@ -1 +0,0 @@ -{"confId":28,"seeds":["{\"url\":\"https://www.amazon.com/bridge-across-time-myths-history/dp/0855000449/ref\\u003dsr_1_1/186-6687480-5099813?ie\\u003dUTF8\\u0026keywords\\u003d0855000449\\u0026qid\\u003d1434663708\\u0026sr\\u003d8-1\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"0855000449\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/product/8479538562/ref\\u003dsr_1_9?ie\\u003dUTF8\\u0026qid\\u003d47073302\\u0026sr\\u003d8-1\\u0026keywords\\u003dnull\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"8479538562\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/product/1619619172/ref\\u003ds9_simh_gw_p63_d3_i3?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-1\\u0026pf_rd_r\\u003d80956L21ZP4Y9DRF74Z5\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d988263899\\u0026pf_rd_i\\u003d891862\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1619619172\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/1857751507/ref\\u003ds9_simh_gw_p74_d2_i8?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-2\\u0026pf_rd_r\\u003d81TU9DQLRW1TQM56LWW3\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d156743226\\u0026pf_rd_i\\u003d729862\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1857751507\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/product/1481714317/ref\\u003ds9_simh_gw_p107_d0_i4?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-1\\u0026pf_rd_r\\u003dKB37RY3JE5HKQ5G4630T\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d331346081\\u0026pf_rd_i\\u003d575468\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1481714317\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/product/1403775141/ref\\u003dcm_cr_pr_product_top\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1403775141\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Letters-Amelia-Diana-Turner-Forte/dp/0533157684/ref\\u003dsr_1_1/184-7208233-4184259?ie\\u003dUTF8\\u0026keywords\\u003d9780533157686\\u0026qid\\u003d1434675492\\u0026sr\\u003d8-1\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"0533157684\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/product/1591022320/ref\\u003ds9_simh_gw_p63_d3_i5?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-2\\u0026pf_rd_r\\u003dWVP9OE92HD77NSJXQZIL\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d556903239\\u0026pf_rd_i\\u003d594333\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1591022320\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Medieval-Early-Modern-Times-Janet/dp/9994609912/ref\\u003dsr_1_1/191-6299628-5905209?ie\\u003dUTF8\\u0026keywords\\u003d9789994609918\\u0026qid\\u003d1434432677\\u0026sr\\u003d8-1\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"9994609912\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/gp/product/1493574175/ref\\u003ds9_simh_gw_p74_d2_i6?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-3\\u0026pf_rd_r\\u003d1OZ6A0RNM4S8JJ01245S\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d124388178\\u0026pf_rd_i\\u003d596200\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1493574175\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/1598598635/ref\\u003ds9_ri_gw_g201_i7?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-3\\u0026pf_rd_r\\u003d7O715KMM6Y744QTQ4LIU\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d673955261\\u0026pf_rd_i\\u003d649226\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1598598635\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Escape-Tibet-Nick-Gray/dp/095715190X/ref\\u003dsr_1_1/175-7416659-0137139?ie\\u003dUTF8\\u0026keywords\\u003d9780957151901\\u0026qid\\u003d1434508924\\u0026sr\\u003d8-1\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"095715190X\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Man-About-House-Definitive-Companion-ebook/dp/B005AKCFXA\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"B005AKCFXA\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/gp/product/112067641X/ref\\u003ds9_ri_gw_g201_i5?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-3\\u0026pf_rd_r\\u003d0SIZV93I9RQ1669EES6L\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d504839570\\u0026pf_rd_i\\u003d161781\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"112067641X\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Unnatural-Death-Pedigree-Dorothy-Sayers/dp/B005TCM1X8/ref\\u003dsr_1_1/188-6111852-5942804?ie\\u003dUTF8\\u0026keywords\\u003d9780450020988\\u0026qid\\u003d1434450513\\u0026sr\\u003d8-1\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"B005TCM1X8\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/James-Castle-Common-Place-0970425716/dp/B002J813MA/ref\\u003dsr_1_2/176-6368479-5598662?ie\\u003dUTF8\\u0026keywords\\u003d0970425716\\u0026qid\\u003d1434422520\\u0026sr\\u003d8-2\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"B002J813MA\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/1940516099/ref\\u003ds9_ri_gw_g201_i2?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-1\\u0026pf_rd_r\\u003dR8JQ4JMW6P3SOJL36M9M\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d535308337\\u0026pf_rd_i\\u003d184331\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1940516099\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/product/1240463456/ref\\u003ds9_ri_gw_g201_i3?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-3\\u0026pf_rd_r\\u003dBL66W75USY907YG001QF\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d720488364\\u0026pf_rd_i\\u003d676890\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1240463456\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/CARSON-DELLOSA-BRIDGE-ORANGE-4-5/dp/1932210652/ref\\u003dsr_1_1/186-9625969-7817661?ie\\u003dUTF8\\u0026keywords\\u003d1932210652\\u0026qid\\u003d1434486504\\u0026sr\\u003d8-1\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1932210652\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/product/B00898M8X2/ref\\u003ds9_simh_gw_p74_d2_i6?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-2\\u0026pf_rd_r\\u003d6RTM9ED4621X306QR2Y3\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d205846301\\u0026pf_rd_i\\u003d110484\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"B00898M8X2\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/product/1597549045/ref\\u003ds9_simh_gw_p63_d3_i10?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-2\\u0026pf_rd_r\\u003dOLWI416TK382276FRI8S\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d971991139\\u0026pf_rd_i\\u003d747799\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1597549045\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Betrayal-Empty-Coffin-Novel/dp/1469216868/ref\\u003dsr_1_1/175-7820384-1782744?ie\\u003dUTF8\\u0026keywords\\u003d1469216868\\u0026qid\\u003d1434594698\\u0026sr\\u003d8-1\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1469216868\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/American-Police-Systems-Raymond-Fosdick/dp/B001KUVGPW/ref\\u003dsr_1_1/176-3370848-7400366?ie\\u003dUTF8\\u0026keywords\\u003d0875859097\\u0026qid\\u003d1434597754\\u0026sr\\u003d8-1\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"B001KUVGPW\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/gp/product/B00KDV1514/ref\\u003dsr_1_1?ie\\u003dUTF8\\u0026qid\\u003d68571520\\u0026sr\\u003d8-1\\u0026keywords\\u003dnull\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"B00KDV1514\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/product/1599928825/ref\\u003ds9_ri_gw_g201_i7?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-1\\u0026pf_rd_r\\u003d4TLP90H57YN3J9USNZS0\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d206214160\\u0026pf_rd_i\\u003d410077\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1599928825\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Obeah-Bible-L-W-Laurence/dp/1456472992/184-5031071-1689052?ie\\u003dUTF8\\u0026ref_\\u003ds9_simh_gw_p107_d0_i9\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1456472992\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/product/B00XJOGUL2/ref\\u003ds9_simh_gw_p63_d3_i4?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-1\\u0026pf_rd_r\\u003d6K4IS5NT23VSY8RK09SM\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d971785392\\u0026pf_rd_i\\u003d254059\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"metaCategId\\\":625,\\\"leafCategId\\\":43479,\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"B00XJOGUL2\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/product/0735841594/ref\\u003ds9_simh_gw_p74_d2_i2?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-3\\u0026pf_rd_r\\u003d4S47E7MUXUTX2OAS03M7\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d858738605\\u0026pf_rd_i\\u003d329711\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"0735841594\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/gp/product/1401246125/ref\\u003ds9_ri_gw_g201_i1?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-2\\u0026pf_rd_r\\u003dD5ZSN8FGJFRE1P6P06FN\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d986128032\\u0026pf_rd_i\\u003d431901\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1401246125\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/gp/product/0531233545/ref\\u003dcm_cr_pr_product_top\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"0531233545\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Magpies-Psychological-Thriller-Mark-Edwards/dp/1483911896/ref\\u003dsr_1_1/186-3277371-2912266?ie\\u003dUTF8\\u0026keywords\\u003d9781483911892\\u0026qid\\u003d1434440841\\u0026sr\\u003d8-1\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1483911896\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/gp/product/149936816X/ref\\u003ds9_ri_gw_g201_i9?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-3\\u0026pf_rd_r\\u003dA2G0NAMUI8Y2SLIOG6K1\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d206377910\\u0026pf_rd_i\\u003d192380\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"149936816X\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/gp/product/0321993306/ref\\u003ds9_simh_gw_p74_d2_i9?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-2\\u0026pf_rd_r\\u003dP49OLGZR6428DSLYD41K\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d464469450\\u0026pf_rd_i\\u003d713302\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"0321993306\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Arcana-Archives-Catalogo-Febbraio-9-Settembre/dp/8860604222/ref\\u003dsr_1_1/191-2294536-5098349?ie\\u003dUTF8\\u0026keywords\\u003d9788860604224\\u0026qid\\u003d1434516786\\u0026sr\\u003d8-1\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"8860604222\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/1287198279/ref\\u003dcm_cr_pr_product_top\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1287198279\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/gp/product/B005AW9VPI/ref\\u003ds9_simh_gw_p79_d17_i9?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-1\\u0026pf_rd_r\\u003dSH241RMHIXZ0P4OG3QO7\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d946293479\\u0026pf_rd_i\\u003d418397\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"B005AW9VPI\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/product/1844651258/ref\\u003dsr_1_2?ie\\u003dUTF8\\u0026qid\\u003d70179092\\u0026sr\\u003d8-1\\u0026keywords\\u003dnull\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1844651258\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/1484105176/ref\\u003dsr_1_4?ie\\u003dUTF8\\u0026qid\\u003d40145443\\u0026sr\\u003d8-1\\u0026keywords\\u003dnull\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1484105176\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Monday-Morning-Leadership-Valerie-Sokolosky/dp/097464031X/ref\\u003dsr_1_1/192-6857494-6582456?ie\\u003dUTF8\\u0026keywords\\u003d9780974640310\\u0026qid\\u003d1434634974\\u0026sr\\u003d8-1\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"097464031X\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/1288670427/ref\\u003ds9_simh_gw_p79_d17_i5?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-2\\u0026pf_rd_r\\u003dEF7S9AY28SM61TL6P5XR\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d377137900\\u0026pf_rd_i\\u003d382960\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1288670427\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/B00K8UNGGW/ref\\u003dcm_cr_pr_product_top\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"B00K8UNGGW\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Watcher-Another-World-J-Wilson/dp/187167672X/ref\\u003dsr_1_1/179-3023112-0477816?ie\\u003dUTF8\\u0026keywords\\u003d9781871676723\\u0026qid\\u003d1434542107\\u0026sr\\u003d8-1\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"187167672X\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Mad-Black-Lady-Wanda-Coleman/dp/0876854129/ref\\u003dsr_1_1/182-7097983-9105503?ie\\u003dUTF8\\u0026keywords\\u003d0876854129\\u0026qid\\u003d1434701480\\u0026sr\\u003d8-1\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"0876854129\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Every-Word-Fist-Amelia-Garcia/dp/146620818X\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"146620818X\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/product/B00873B0CO/ref\\u003ds9_simh_gw_p74_d2_i5?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-2\\u0026pf_rd_r\\u003d0XY59A109S39P6ID1N23\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d295494213\\u0026pf_rd_i\\u003d478561\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"B00873B0CO\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Springboard-Discovery-Mary-Lou-Lacy/dp/0804235953\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"0804235953\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Aventures-extraordinaires-dAdÃ%C2%83Â%C2%83Â%C2%83Ã%C2%83Â%C2%82Â%C2%83Ã%C2%83Â%C2%82Â%C2%83Ã%C2%83Â%C2%83Â%C2%83Ã%C2%83Â%C2%82Â%C2%82Ã%C2%83Â%C2%82Â%C2%83Ã%C2%83Â%C2%83Â%C2%83Ã%C2%83Â%C2%82Â%C2%83Ã%C2%83Â%C2%82Â%C2%85Ã%C2%83Â%C2%83Â%C2%83Ã%C2%83Â%C2%82Â%C2%82Ã%C2%83Â%C2%83Â%C2%82Ã%C2%83Â%C2%82Ã%C2%82¡le-Blanc-Sec/dp/229032096X\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"229032096X\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/product/1480512796/ref\\u003ds9_ri_gw_g201_i4?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-3\\u0026pf_rd_r\\u003dK4N7A64LBYP7TXSR9W49\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d681535512\\u0026pf_rd_i\\u003d230294\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1480512796\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/crise-conscience-europeenne-1680-1715-French/dp/221300613X/ref\\u003dsr_1_1/189-4114609-4176061?ie\\u003dUTF8\\u0026keywords\\u003d9782213006130\\u0026qid\\u003d1434473448\\u0026sr\\u003d8-1\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"221300613X\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/1743170246/ref\\u003dcm_cr_pr_product_top\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1743170246\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/1249925037/ref\\u003ds9_simh_gw_p63_d3_i2?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-3\\u0026pf_rd_r\\u003d1ZX7BOB71HEJOS512320\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d213504420\\u0026pf_rd_i\\u003d514621\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1249925037\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Lifetime-Volunteer-Frates-Joan-Gilmore/dp/1885596499/ref\\u003dsr_1_1/184-6208647-5211900?ie\\u003dUTF8\\u0026keywords\\u003d1885596499\\u0026qid\\u003d1434664139\\u0026sr\\u003d8-1\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1885596499\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/product/1507597142/ref\\u003ds9_simh_gw_p107_d0_i8?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-1\\u0026pf_rd_r\\u003dG88Q8FMLD0RS001F6STG\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d192898073\\u0026pf_rd_i\\u003d725640\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1507597142\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/gp/product/0691614385/ref\\u003dsr_1_7?ie\\u003dUTF8\\u0026qid\\u003d11874892\\u0026sr\\u003d8-1\\u0026keywords\\u003dnull\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"0691614385\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/gp/product/150531089X/ref\\u003dsr_1_8?ie\\u003dUTF8\\u0026qid\\u003d93416624\\u0026sr\\u003d8-1\\u0026keywords\\u003dnull\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"150531089X\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/gp/product/1591859581/ref\\u003ds9_simh_gw_p74_d2_i7?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-1\\u0026pf_rd_r\\u003dD89JO942AP0BGI8VHY6Z\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d259933109\\u0026pf_rd_i\\u003d594179\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1591859581\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/product/B0077D8O60/ref\\u003dcm_cr_pr_product_top\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"B0077D8O60\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/gp/product/0957401515/ref\\u003ds9_simh_gw_p63_d3_i2?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-3\\u0026pf_rd_r\\u003dP68SFUUGN08H8EL54714\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d948202552\\u0026pf_rd_i\\u003d899110\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"0957401515\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/3849554147/ref\\u003ds9_simh_gw_p79_d17_i8?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-2\\u0026pf_rd_r\\u003d1QMUT47S5N1OP87AM79G\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d664853631\\u0026pf_rd_i\\u003d825108\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"3849554147\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Regulating-Pesticides-Commission-Natural-Resources/dp/0309029465/ref\\u003dsr_1_1/186-2886194-1044035?ie\\u003dUTF8\\u0026keywords\\u003d0309029465\\u0026qid\\u003d1434700978\\u0026sr\\u003d8-1\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"0309029465\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/AutoCAD-2006-Tutorial-First-Fundamentals/dp/1585032301\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1585032301\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/product/1493162667/ref\\u003ds9_simh_gw_p63_d3_i8?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-1\\u0026pf_rd_r\\u003d5B1N58OK9DGNSXAYO648\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d490117977\\u0026pf_rd_i\\u003d302914\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1493162667\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/gp/product/1482532700/ref\\u003ds9_simh_gw_p79_d17_i4?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-1\\u0026pf_rd_r\\u003d2A85UB174G4QB67Q6W4E\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d469121724\\u0026pf_rd_i\\u003d652299\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1482532700\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/product/1289164118/ref\\u003dsr_1_2?ie\\u003dUTF8\\u0026qid\\u003d18547913\\u0026sr\\u003d8-1\\u0026keywords\\u003dnull\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1289164118\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/1275841597/ref\\u003dsr_1_6?ie\\u003dUTF8\\u0026qid\\u003d83157364\\u0026sr\\u003d8-1\\u0026keywords\\u003dnull\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1275841597\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/How-Maximize-Your-Profit-maximize/dp/1505906075\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1505906075\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/1436679060/ref\\u003ds9_simh_gw_p79_d17_i1?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-2\\u0026pf_rd_r\\u003dJ9SOI248X12J3TS2DL0V\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d526717715\\u0026pf_rd_i\\u003d369502\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1436679060\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Plant-Molecular-Biology-Essential-Techniques/dp/0471972681/ref\\u003dsr_1_1/190-9297182-6303650?ie\\u003dUTF8\\u0026keywords\\u003d9780471972686\\u0026qid\\u003d1434705995\\u0026sr\\u003d8-1\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"0471972681\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Europe-Exporters-Handbook-Jenner/dp/0871965658/ref\\u003dsr_1_1/189-3435491-9141552?ie\\u003dUTF8\\u0026keywords\\u003d9780871965653\\u0026qid\\u003d1434701852\\u0026sr\\u003d8-1\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"0871965658\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/product/1288527004/ref\\u003ds9_simh_gw_p79_d17_i2?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-1\\u0026pf_rd_r\\u003dL676IU8SE79EB28460T4\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d839567868\\u0026pf_rd_i\\u003d952718\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1288527004\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Groups-Practice-Marianne-Schneider-Hardcover/dp/0534367453/ref\\u003dsr_1_1/186-0267917-7112209?ie\\u003dUTF8\\u0026keywords\\u003d0534367453\\u0026qid\\u003d1434879089\\u0026sr\\u003d8-1\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"0534367453\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/product/161034958X/ref\\u003ds9_simh_gw_p79_d17_i3?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-2\\u0026pf_rd_r\\u003dLDIXFU564R4V24ZK6SN2\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d473453365\\u0026pf_rd_i\\u003d771807\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"161034958X\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/product/B00049QL5U/ref\\u003ds9_simh_gw_p107_d0_i2?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-2\\u0026pf_rd_r\\u003dY564P8EU9SRSEXNYP77G\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d561884715\\u0026pf_rd_i\\u003d312289\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"B00049QL5U\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/product/1618852183/ref\\u003ds9_simh_gw_p107_d0_i8?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-3\\u0026pf_rd_r\\u003dKM0Q6544JP28U5676KZF\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d974311606\\u0026pf_rd_i\\u003d652689\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1618852183\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Models-Dermatology-Vol/dp/3805547617\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"3805547617\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/gp/product/0684859726/ref\\u003ds9_simh_gw_p79_d17_i7?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-2\\u0026pf_rd_r\\u003d727ES4S50NPD7M8F4XLO\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d475467204\\u0026pf_rd_i\\u003d897737\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"0684859726\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/gp/product/1449788386/ref\\u003dsr_1_3?ie\\u003dUTF8\\u0026qid\\u003d34593636\\u0026sr\\u003d8-1\\u0026keywords\\u003dnull\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1449788386\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/1622125290/ref\\u003ds9_simh_gw_p107_d0_i7?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-1\\u0026pf_rd_r\\u003d46Q876OOMM3D6XSDG7VM\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d954847296\\u0026pf_rd_i\\u003d374067\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1622125290\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/gp/product/1494559447/ref\\u003ds9_simh_gw_p79_d17_i8?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-1\\u0026pf_rd_r\\u003dW5LSQ3KS109584JUY4A0\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d694179817\\u0026pf_rd_i\\u003d134867\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1494559447\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Proceedings-1-2-Classical-Association-Wales/dp/1236636929/ref\\u003dsr_1_1/184-1359550-5736018?ie\\u003dUTF8\\u0026keywords\\u003d9781236636928\\u0026qid\\u003d1434649299\\u0026sr\\u003d8-1\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1236636929\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/gp/product/1287003001/ref\\u003dcm_cr_pr_product_top\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1287003001\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/gp/product/1249180309/ref\\u003dsr_1_10?ie\\u003dUTF8\\u0026qid\\u003d32633819\\u0026sr\\u003d8-1\\u0026keywords\\u003dnull\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1249180309\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Michelin-Red-Guide-Spain-Portugal/dp/2060063779/ref\\u003dsr_1_1/179-4403527-2002065?ie\\u003dUTF8\\u0026keywords\\u003d9782060063775\\u0026qid\\u003d1434621406\\u0026sr\\u003d8-1\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"2060063779\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Homicidal-Aliens-Other-Disappointments-Invasion/dp/1480518522/ref\\u003dsr_1_1/186-9648743-6858653?ie\\u003dUTF8\\u0026keywords\\u003d1480518522\\u0026qid\\u003d1434660937\\u0026sr\\u003d8-1\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1480518522\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/0984635742/ref\\u003ds9_simh_gw_p79_d17_i4?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-1\\u0026pf_rd_r\\u003d21YWJSFV761872B02I7H\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d175584267\\u0026pf_rd_i\\u003d665215\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"0984635742\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/product/1583335331/ref\\u003ds9_simh_gw_p74_d2_i8?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-1\\u0026pf_rd_r\\u003dPLKAEVA1UJH99O85N5K2\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d919768077\\u0026pf_rd_i\\u003d142182\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1583335331\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/1305256662/ref\\u003ds9_ri_gw_g201_i6?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-2\\u0026pf_rd_r\\u003d8V35F53XPBW54M0TO89S\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d736378767\\u0026pf_rd_i\\u003d803718\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1305256662\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Biological-Rhythm-Research-Sollberger/dp/0444405496/ref\\u003dsr_1_1/184-0842412-0577011?ie\\u003dUTF8\\u0026keywords\\u003d0444405496\\u0026qid\\u003d1434660594\\u0026sr\\u003d8-1\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"0444405496\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/product/0826520235/ref\\u003ds9_simh_gw_p79_d17_i8?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-2\\u0026pf_rd_r\\u003d17N0U1085ISNRS396YHI\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d909993468\\u0026pf_rd_i\\u003d167394\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"0826520235\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/1452089175/ref\\u003ds9_simh_gw_p63_d3_i7?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-2\\u0026pf_rd_r\\u003d9S13UGLB1J3M03X087L7\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d683610611\\u0026pf_rd_i\\u003d897880\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1452089175\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/gp/product/1286720451/ref\\u003ds9_simh_gw_p79_d17_i2?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-2\\u0026pf_rd_r\\u003dE82MQGB807FFVWB3Q44R\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d104104751\\u0026pf_rd_i\\u003d243022\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1286720451\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/product/1499041055/ref\\u003ds9_ri_gw_g201_i9?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-1\\u0026pf_rd_r\\u003dB5S17N4WLLSPDD34YDY8\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d872947608\\u0026pf_rd_i\\u003d644957\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1499041055\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/1288769598/ref\\u003dcm_cr_pr_product_top\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1288769598\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Encyclopaedia-United-States-Spacecraft-Bison/dp/0600500519/ref\\u003dsr_1_1/189-9370660-2213859?ie\\u003dUTF8\\u0026keywords\\u003d9780600500513\\u0026qid\\u003d1434620291\\u0026sr\\u003d8-1\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"0600500519\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/ANTONIO-VIVALDI-Garland-reference-humanities/dp/0824083865\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"0824083865\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/gp/product/1465417117/ref\\u003dsr_1_4?ie\\u003dUTF8\\u0026qid\\u003d16061560\\u0026sr\\u003d8-1\\u0026keywords\\u003dnull\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1465417117\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/gp/product/0813224535/ref\\u003ds9_simh_gw_p74_d2_i6?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-3\\u0026pf_rd_r\\u003dSHQ8F60N372GY56JWF09\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d412949045\\u0026pf_rd_i\\u003d425983\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"0813224535\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/gp/product/1495421996/ref\\u003ds9_ri_gw_g201_i4?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-2\\u0026pf_rd_r\\u003dX9D65SVNFOVU7N8SWF3O\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d165045927\\u0026pf_rd_i\\u003d372797\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1495421996\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/product/3640223098/ref\\u003dcm_cr_pr_product_top\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"3640223098\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Soldier-Official-Guide-Accompany-ITV/dp/B0041CQMJG\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"B0041CQMJG\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/1480031003/ref\\u003ds9_simh_gw_p63_d3_i10?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-2\\u0026pf_rd_r\\u003d0RSF4Y1580S2V22H2JSG\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d328419621\\u0026pf_rd_i\\u003d991785\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1480031003\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/My-Harvest-Home-Celebration-Customs/dp/0937203688/ref\\u003dsr_1_1/185-2501167-9365558?ie\\u003dUTF8\\u0026keywords\\u003d9780937203682\\u0026qid\\u003d1434501536\\u0026sr\\u003d8-1\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"0937203688\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Halfway-Decent-Ernie-Hudson/dp/B001A4YO20\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"B001A4YO20\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/1288916906/ref\\u003ds9_simh_gw_p74_d2_i6?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-3\\u0026pf_rd_r\\u003dONRG59HTI83U545RG01Y\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d669707968\\u0026pf_rd_i\\u003d353895\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1288916906\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/product/1288725108/ref\\u003dsr_1_5?ie\\u003dUTF8\\u0026qid\\u003d97736076\\u0026sr\\u003d8-1\\u0026keywords\\u003dnull\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1288725108\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/product/1462067514/ref\\u003dcm_cr_pr_product_top\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1462067514\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/1290713707/ref\\u003ds9_simh_gw_p107_d0_i8?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-1\\u0026pf_rd_r\\u003dR9AF8E0I0OVS17OZWIL0\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d819156518\\u0026pf_rd_i\\u003d124919\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1290713707\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Het-historische-gedicht-geschiedenis-Vlaanderen/dp/9057592851/ref\\u003dsr_1_1/176-4708134-2081319?ie\\u003dUTF8\\u0026keywords\\u003d9789057592850\\u0026qid\\u003d1434457720\\u0026sr\\u003d8-1\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"9057592851\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/product/1289300151/ref\\u003ds9_simh_gw_p63_d3_i3?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-3\\u0026pf_rd_r\\u003dDW5AQM8981BG14F3OR4U\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d353942470\\u0026pf_rd_i\\u003d642212\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1289300151\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Christines-Picture-Book-Christian-Andersen/dp/0862720893/ref\\u003dsr_1_1/179-3046203-1921448?ie\\u003dUTF8\\u0026keywords\\u003d9780862720896\\u0026qid\\u003d1434426244\\u0026sr\\u003d8-1\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"0862720893\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/product/1249461162/ref\\u003ds9_simh_gw_p79_d17_i3?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-1\\u0026pf_rd_r\\u003d57YUBLM3S5JH10JY6SO4\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d826066547\\u0026pf_rd_i\\u003d907699\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1249461162\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/1246822881/ref\\u003dsr_1_10?ie\\u003dUTF8\\u0026qid\\u003d18489479\\u0026sr\\u003d8-1\\u0026keywords\\u003dnull\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1246822881\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/product/1497903920/ref\\u003dcm_cr_pr_product_top\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1497903920\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/product/1105315800/ref\\u003ds9_simh_gw_p74_d2_i2?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-3\\u0026pf_rd_r\\u003dISLI9GZS5J520O513PK6\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d191025471\\u0026pf_rd_i\\u003d563906\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1105315800\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/gp/product/1249420024/ref\\u003ds9_simh_gw_p63_d3_i6?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-2\\u0026pf_rd_r\\u003dBG6V7MK62E7322D5VS4K\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d335458808\\u0026pf_rd_i\\u003d388332\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1249420024\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/product/1287198848/ref\\u003ds9_simh_gw_p107_d0_i10?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-3\\u0026pf_rd_r\\u003dUATP2780PEBSP3MZ85J9\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d724971579\\u0026pf_rd_i\\u003d737723\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1287198848\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/product/1599152088/ref\\u003ds9_simh_gw_p74_d2_i1?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-1\\u0026pf_rd_r\\u003d1R9QZBS3LHGN2QKTW84D\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d285986720\\u0026pf_rd_i\\u003d167207\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1599152088\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/product/1287183700/ref\\u003ds9_simh_gw_p79_d17_i7?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-3\\u0026pf_rd_r\\u003d4W49A4JP56S6IELDOF3S\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d604578060\\u0026pf_rd_i\\u003d666380\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1287183700\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/product/1470817403/ref\\u003ds9_simh_gw_p63_d3_i10?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-1\\u0026pf_rd_r\\u003d72AXH8EE5ZQTRY727NNV\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d213474611\\u0026pf_rd_i\\u003d351283\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1470817403\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Sonnets-Facing-Conviction-Christopher-Presfield/dp/1891812203\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1891812203\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Dead-Ringer-Jasmine-Cresswell/dp/B0010BSZ5U/ref\\u003dsr_1_1/190-6876153-5019461?ie\\u003dUTF8\\u0026keywords\\u003d1551667126\\u0026qid\\u003d1434458583\\u0026sr\\u003d8-1\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"B0010BSZ5U\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/1933660996/ref\\u003dcm_cr_pr_product_top\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1933660996\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Sherington-Fiefs-Fields-Buckinghamshire-Village/dp/0521046378/ref\\u003dsr_1_1/182-5625910-7791542?ie\\u003dUTF8\\u0026keywords\\u003d9780521046374\\u0026qid\\u003d1434591978\\u0026sr\\u003d8-1\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"0521046378\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/gp/product/0452297060/ref\\u003ds9_ri_gw_g201_i6?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-3\\u0026pf_rd_r\\u003d9WQLGU3K20TZ97OLNE9N\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d817830578\\u0026pf_rd_i\\u003d592054\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"0452297060\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Believers-Pocket-Companion-Needful-Sinners/dp/1889058092/ref\\u003dsr_1_1/181-8057383-5400821?ie\\u003dUTF8\\u0026keywords\\u003d1889058092\\u0026qid\\u003d1434591575\\u0026sr\\u003d8-1\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1889058092\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/gp/product/1288686684/ref\\u003ds9_simh_gw_p107_d0_i5?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-2\\u0026pf_rd_r\\u003d6G636UL89WPEJRNWM864\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d863114753\\u0026pf_rd_i\\u003d127597\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1288686684\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/product/1441968091/ref\\u003ds9_simh_gw_p107_d0_i7?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-1\\u0026pf_rd_r\\u003dVD3N8XQHIN218881396G\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d591946161\\u0026pf_rd_i\\u003d784879\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1441968091\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Illustrated-Skating-Dictionary-Young-People/dp/013451260X/ref\\u003dsr_1_1/187-7738327-4220041?ie\\u003dUTF8\\u0026keywords\\u003d9780134512600\\u0026qid\\u003d1434671356\\u0026sr\\u003d8-1\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"013451260X\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/gp/product/1286427355/ref\\u003ds9_simh_gw_p63_d3_i4?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-2\\u0026pf_rd_r\\u003dG08BDQF87W390SN0VKSY\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d609639660\\u0026pf_rd_i\\u003d873910\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1286427355\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/1592999107/ref\\u003ds9_simh_gw_p63_d3_i10?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-3\\u0026pf_rd_r\\u003d0GZEX2Z031943Y98PS2O\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d207358316\\u0026pf_rd_i\\u003d646461\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1592999107\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/product/0971487421/ref\\u003ds9_simh_gw_p79_d17_i4?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-3\\u0026pf_rd_r\\u003dVP2GYS4S48L27YR91NGX\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d288057837\\u0026pf_rd_i\\u003d811217\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"0971487421\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Dakine-Traction-12-25-Inch-Black-Smoke/dp/B009HYI5U2/ref\\u003dsr_1_1/190-3075409-4147065?ie\\u003dUTF8\\u0026keywords\\u003d0610934775921\\u0026qid\\u003d1434572027\\u0026sr\\u003d8-1\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"B009HYI5U2\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/gp/product/1505465214/ref\\u003ds9_simh_gw_p79_d17_i10?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-2\\u0026pf_rd_r\\u003dS88VYH0JSN0WV58N763S\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d469060850\\u0026pf_rd_i\\u003d923112\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1505465214\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/gp/product/1249477794/ref\\u003ds9_ri_gw_g201_i3?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-2\\u0026pf_rd_r\\u003d1YEA01A3X8LEV2A1XUYL\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d163201523\\u0026pf_rd_i\\u003d432311\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1249477794\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/gp/product/1439101515/ref\\u003dsr_1_10?ie\\u003dUTF8\\u0026qid\\u003d41353122\\u0026sr\\u003d8-1\\u0026keywords\\u003dnull\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1439101515\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/gp/product/0871522748/ref\\u003ds9_simh_gw_p63_d3_i8?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-3\\u0026pf_rd_r\\u003dM2XLTA5E84OZ9WW38KL1\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d173836115\\u0026pf_rd_i\\u003d585725\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"0871522748\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Guide-Florida-legal-research-Stupski/dp/0327155868/ref\\u003dsr_1_1/178-6731625-0968554?ie\\u003dUTF8\\u0026keywords\\u003d0327155868\\u0026qid\\u003d1434593715\\u0026sr\\u003d8-1\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"0327155868\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/product/0735341257/ref\\u003dcm_cr_pr_product_top\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"0735341257\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/1287251641/ref\\u003ds9_simh_gw_p63_d3_i1?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-1\\u0026pf_rd_r\\u003d1KEUJ6L7GYD56MD665L5\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d508856645\\u0026pf_rd_i\\u003d515955\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1287251641\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Points-2012-2013-Handbook-Brigade-Midshipmen/dp/161251149X/ref\\u003dsr_1_1/191-7390079-6376012?ie\\u003dUTF8\\u0026keywords\\u003d9781612511498\\u0026qid\\u003d1434621407\\u0026sr\\u003d8-1\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"161251149X\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/product/9050634788/ref\\u003dcm_cr_pr_product_top\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"9050634788\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/gp/product/1406829730/ref\\u003ds9_simh_gw_p63_d3_i8?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-3\\u0026pf_rd_r\\u003d86Y0VR3VTP14529B9TK2\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d516175225\\u0026pf_rd_i\\u003d152667\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1406829730\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/product/1120716756/ref\\u003dcm_cr_pr_product_top\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1120716756\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/gp/product/1449781837/ref\\u003ds9_ri_gw_g201_i9?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-2\\u0026pf_rd_r\\u003dQI0MOKJ0782RNRSXO3EJ\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d853693070\\u0026pf_rd_i\\u003d466575\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1449781837\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Omnibus-Pusher-Kings-Ransom-Money/dp/1407221035/ref\\u003dsr_1_1/187-4173899-5264528?ie\\u003dUTF8\\u0026keywords\\u003d9781407221038\\u0026qid\\u003d1434673964\\u0026sr\\u003d8-1\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1407221035\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/gp/product/1500892181/ref\\u003ds9_simh_gw_p107_d0_i1?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-3\\u0026pf_rd_r\\u003d3S5TF02AB6MRAFME7AKJ\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d572077303\\u0026pf_rd_i\\u003d484731\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1500892181\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/product/1847975313/ref\\u003ds9_ri_gw_g201_i9?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-3\\u0026pf_rd_r\\u003dAJO40FSP5P875338JO26\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d321291056\\u0026pf_rd_i\\u003d704134\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1847975313\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/1249000106/ref\\u003ds9_simh_gw_p74_d2_i4?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-2\\u0026pf_rd_r\\u003dIY3AAVY7WEF77SSKDY0S\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d124726089\\u0026pf_rd_i\\u003d902185\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1249000106\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Golden-Dawn-Journal-Book-Llewellyns/dp/1567188516\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1567188516\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Wee-Pals-Knowledge-Morrie-Turner/dp/0451058003\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"0451058003\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Sourcebook-Public-Record-Providers-Investigative/dp/1879792060\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1879792060\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/World-Omnibus-Books-Bertie-Webster/dp/0216883857/ref\\u003dsr_1_1/185-2474915-1241732?ie\\u003dUTF8\\u0026keywords\\u003d9780216883857\\u0026qid\\u003d1434671154\\u0026sr\\u003d8-1\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"0216883857\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/product/0252080238/ref\\u003ds9_simh_gw_p107_d0_i9?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-1\\u0026pf_rd_r\\u003dFGQSTXT42QRKQ4UL37BL\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d956630874\\u0026pf_rd_i\\u003d262703\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"0252080238\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Adjustment-Equity-Malaysia-Development-Studies/dp/9264136010/ref\\u003dsr_1_1/181-7587893-0591704?ie\\u003dUTF8\\u0026keywords\\u003d9264136010\\u0026qid\\u003d1434613893\\u0026sr\\u003d8-1\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"9264136010\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/product/1411672577/ref\\u003dsr_1_1?ie\\u003dUTF8\\u0026qid\\u003d63432192\\u0026sr\\u003d8-1\\u0026keywords\\u003dnull\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1411672577\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/product/3849566331/ref\\u003ds9_simh_gw_p74_d2_i6?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-1\\u0026pf_rd_r\\u003dSV6SV2689672R60MPXPM\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d618760996\\u0026pf_rd_i\\u003d491874\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"3849566331\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/product/1249325463/ref\\u003ds9_simh_gw_p74_d2_i7?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-1\\u0026pf_rd_r\\u003d3300UTE61DSF42Z24YR6\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d290302597\\u0026pf_rd_i\\u003d909049\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1249325463\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Through-Dark-Labyrinth-Biography-Lawrence/dp/0712666788/ref\\u003dsr_1_1/192-9529270-4371764?ie\\u003dUTF8\\u0026keywords\\u003d9780712666787\\u0026qid\\u003d1434616188\\u0026sr\\u003d8-1\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"0712666788\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/gp/product/127583633X/ref\\u003dcm_cr_pr_product_top\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"127583633X\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Maryland-Replacement-Business-Regulation---Business/dp/0327066628\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"0327066628\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Mysterious-Chills-Thrills-Laura-Hickey/dp/097490130X/ref\\u003dsr_1_1/178-6173788-1828330?ie\\u003dUTF8\\u0026keywords\\u003d9780974901305\\u0026qid\\u003d1434644270\\u0026sr\\u003d8-1\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"097490130X\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Danzig-American-Recordings-74321-23681/dp/B000026GYI/ref\\u003dsr_1_1/182-3883208-9254143?ie\\u003dUTF8\\u0026keywords\\u003d0743212368123\\u0026qid\\u003d1434534995\\u0026sr\\u003d8-1\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"B000026GYI\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/gp/product/1634284798/ref\\u003ds9_simh_gw_p107_d0_i6?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-2\\u0026pf_rd_r\\u003d3895AHWRBT66Z1P9QS0O\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d493371621\\u0026pf_rd_i\\u003d955857\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1634284798\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Auto-Ventshade-194719-Ventvisor-Deflector/dp/B000CFAV8O\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"B000CFAV8O\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/1936744279/ref\\u003ds9_ri_gw_g201_i1?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-1\\u0026pf_rd_r\\u003dINH4U6EIY9P1JLL2A096\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d691439560\\u0026pf_rd_i\\u003d121424\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1936744279\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/1433996014/ref\\u003ds9_simh_gw_p74_d2_i2?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-2\\u0026pf_rd_r\\u003d948GM1R0U4K60I8WI58U\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d727906654\\u0026pf_rd_i\\u003d590985\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1433996014\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Misterio-Metru-Mystery-Bionicle-Spanish/dp/8497632478/ref\\u003dsr_1_1/187-5126550-6996742?ie\\u003dUTF8\\u0026keywords\\u003d9788497632478\\u0026qid\\u003d1434450671\\u0026sr\\u003d8-1\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"8497632478\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/0941613453/ref\\u003dcm_cr_pr_product_top\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"0941613453\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/1480143405/ref\\u003dsr_1_9?ie\\u003dUTF8\\u0026qid\\u003d3814688\\u0026sr\\u003d8-1\\u0026keywords\\u003dnull\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1480143405\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/gp/product/B00GS339YC/ref\\u003ds9_simh_gw_p74_d2_i1?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-3\\u0026pf_rd_r\\u003d1H1S8S7P7AYN485A74B8\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d544362868\\u0026pf_rd_i\\u003d230478\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"metaCategId\\\":11700,\\\"leafCategId\\\":79630,\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"B00GS339YC\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/product/B005T820VK/ref\\u003ds9_simh_gw_p74_d2_i7?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-3\\u0026pf_rd_r\\u003dF3M98S0T37EEOS86X0SR\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d477234694\\u0026pf_rd_i\\u003d588761\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"metaCategId\\\":12576,\\\"leafCategId\\\":181877,\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"B005T820VK\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/gp/product/B00LMKZ5PM/ref\\u003dcm_cr_pr_product_top\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"metaCategId\\\":99,\\\"leafCategId\\\":166830,\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"B00LMKZ5PM\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Microsoft-Excel-Quick-Source-Guide/dp/1932104119/ref\\u003dsr_1_1/179-5887599-8112940?ie\\u003dUTF8\\u0026keywords\\u003d1932104119\\u0026qid\\u003d1434663197\\u0026sr\\u003d8-1\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1932104119\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/product/B00NODYOP2/ref\\u003ds9_simh_gw_p79_d17_i2?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-2\\u0026pf_rd_r\\u003dD1UKURZU7T0RSZDL9Z08\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d102955252\\u0026pf_rd_i\\u003d264326\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"B00NODYOP2\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Forests-What-Are-Andy-Owen/dp/0431023689/ref\\u003dsr_1_1/175-1909095-8188046?ie\\u003dUTF8\\u0026keywords\\u003d9780431023687\\u0026qid\\u003d1434516840\\u0026sr\\u003d8-1\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"0431023689\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Hannahs-brave-year-Rhoda-Wooldridge/dp/083090333X\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"083090333X\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Chemistry-Dummies-Turtleback-Library-Prebound/dp/061391550X/ref\\u003dsr_1_1/182-3704276-6074404?ie\\u003dUTF8\\u0026keywords\\u003d9780613915502\\u0026qid\\u003d1434684306\\u0026sr\\u003d8-1\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"061391550X\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Immigration-secrets-through-immigration-more--/dp/0964001314\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"0964001314\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Adapted-Tests-Prentice-Science-Explorer/dp/0131665979/ref\\u003dsr_1_1/188-0786267-7923645?ie\\u003dUTF8\\u0026keywords\\u003d0131665979\\u0026qid\\u003d1434704033\\u0026sr\\u003d8-1\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"0131665979\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Expense-Control-Productivity-Retailing-profit/dp/0867305436/ref\\u003dsr_1_1/191-1454825-6348502?ie\\u003dUTF8\\u0026keywords\\u003d0867305436\\u0026qid\\u003d1434627013\\u0026sr\\u003d8-1\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"0867305436\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/1455528854/ref\\u003dsr_1_6?ie\\u003dUTF8\\u0026qid\\u003d62509797\\u0026sr\\u003d8-1\\u0026keywords\\u003dnull\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1455528854\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/product/0987935801/ref\\u003dsr_1_6?ie\\u003dUTF8\\u0026qid\\u003d64553029\\u0026sr\\u003d8-1\\u0026keywords\\u003dnull\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"0987935801\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Lucy-Montgomery-Short-Stories-1905-1906/dp/1781392420/ref\\u003dsr_1_1/179-8969282-1218322?ie\\u003dUTF8\\u0026keywords\\u003d1781392420\\u0026qid\\u003d1434444399\\u0026sr\\u003d8-1\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1781392420\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/product/B00DV5WTX2/ref\\u003ds9_simh_gw_p79_d17_i4?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-2\\u0026pf_rd_r\\u003dD6W0KS18E69NEYB18001\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d815214806\\u0026pf_rd_i\\u003d447967\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"B00DV5WTX2\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/product/1461377293/ref\\u003ds9_ri_gw_g201_i10?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-3\\u0026pf_rd_r\\u003d030864T4Q4066704UZ7E\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d592325671\\u0026pf_rd_i\\u003d959581\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1461377293\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/product/1287021905/ref\\u003ds9_simh_gw_p63_d3_i9?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-1\\u0026pf_rd_r\\u003d4HXDWF45T32MD1D3S8GE\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d385879740\\u0026pf_rd_i\\u003d129580\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1287021905\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/1141127326/ref\\u003ds9_simh_gw_p63_d3_i2?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-2\\u0026pf_rd_r\\u003dBM8080B46NGFUSXBNDE1\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d872809181\\u0026pf_rd_i\\u003d473326\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1141127326\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/gp/product/1481755307/ref\\u003ds9_ri_gw_g201_i4?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-1\\u0026pf_rd_r\\u003d5ROK0FPJ6HH04V0O637L\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d115978925\\u0026pf_rd_i\\u003d569414\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1481755307\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/gp/product/1249332729/ref\\u003ds9_simh_gw_p107_d0_i9?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-1\\u0026pf_rd_r\\u003dVEX309MH2YPAS2X5SX95\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d373201442\\u0026pf_rd_i\\u003d192379\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1249332729\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/gp/product/B000FVQLY8/ref\\u003ds9_simh_gw_p79_d17_i4?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-3\\u0026pf_rd_r\\u003dZSVSV5L0Z5SG5D2ZF9IS\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d168856631\\u0026pf_rd_i\\u003d247360\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"B000FVQLY8\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/gp/product/1484914317/ref\\u003ds9_simh_gw_p74_d2_i4?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-3\\u0026pf_rd_r\\u003dGXDNY3K3KAPF1S08AYRO\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d775700064\\u0026pf_rd_i\\u003d704233\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1484914317\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/The-Wild-Dogs/dp/0947072535\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"0947072535\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/product/1489558047/ref\\u003ds9_ri_gw_g201_i6?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-1\\u0026pf_rd_r\\u003dQHWMSIYZ99493U8IN9E5\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d192725230\\u0026pf_rd_i\\u003d993622\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1489558047\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/product/0866906339/ref\\u003ds9_simh_gw_p107_d0_i4?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-2\\u0026pf_rd_r\\u003d7G9HE2IF43JWM74SZY2Z\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d930027883\\u0026pf_rd_i\\u003d439608\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"0866906339\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Wedding-Book-Experts-Planning-Day-Your-ebook/dp/B006MGDJB8\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"B006MGDJB8\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Dios-Habla-Hoy-Spanish-Edition/dp/1585161187/ref\\u003dsr_1_1/176-1062857-7907332?ie\\u003dUTF8\\u0026keywords\\u003d9781585161188\\u0026qid\\u003d1434437626\\u0026sr\\u003d8-1\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1585161187\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/product/551859366X/ref\\u003ds9_simh_gw_p63_d3_i8?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-2\\u0026pf_rd_r\\u003d7TW10D9B9W7501800JDZ\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d239708876\\u0026pf_rd_i\\u003d845749\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"551859366X\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/gp/product/B000TK7OZA/ref\\u003dsr_1_3?ie\\u003dUTF8\\u0026qid\\u003d41558171\\u0026sr\\u003d8-1\\u0026keywords\\u003dnull\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"B000TK7OZA\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/product/B00C77DS58/ref\\u003dsr_1_2?ie\\u003dUTF8\\u0026qid\\u003d70641569\\u0026sr\\u003d8-1\\u0026keywords\\u003dnull\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"metaCategId\\\":11700,\\\"leafCategId\\\":79654,\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"B00C77DS58\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/product/1499698259/ref\\u003ds9_simh_gw_p63_d3_i4?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-2\\u0026pf_rd_r\\u003d66W4059MU38M8JZ3RFA3\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d417187228\\u0026pf_rd_i\\u003d899409\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1499698259\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Souvenir-Guide-Fourth-Botanic-Gardens/dp/1842464140\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1842464140\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Beasts-Gor-John-Norman/dp/B001AP3KH4/ref\\u003dsr_1_1/190-0932607-6814544?ie\\u003dUTF8\\u0026keywords\\u003d0352303964\\u0026qid\\u003d1434627148\\u0026sr\\u003d8-1\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"B001AP3KH4\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Off-Balance-Purpose-Embrace-Uncertainty/dp/1608320146/ref\\u003dcm_cr_pr_product_top\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1608320146\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Metal-Vapor-Lasers-Applications-Proceedings/dp/0819405027/ref\\u003dsr_1_1/187-8627002-9363742?ie\\u003dUTF8\\u0026keywords\\u003d9780819405029\\u0026qid\\u003d1434697125\\u0026sr\\u003d8-1\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"0819405027\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Mormon-Temple-Square-Behind-Scenery/dp/0887147585/ref\\u003dsr_1_1/179-6133906-6553235?ie\\u003dUTF8\\u0026keywords\\u003d9780887147586\\u0026qid\\u003d1434458335\\u0026sr\\u003d8-1\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"0887147585\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/product/1249331412/ref\\u003ds9_simh_gw_p79_d17_i7?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-1\\u0026pf_rd_r\\u003dFZYF6X720561ZOQ6Q42O\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d730531613\\u0026pf_rd_i\\u003d644545\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1249331412\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Hablar-Idioma-Cliente-Healthcare-Professionals/dp/0966022572/ref\\u003dsr_1_1/180-5338436-6086643?ie\\u003dUTF8\\u0026keywords\\u003d0966022572\\u0026qid\\u003d1434505188\\u0026sr\\u003d8-1\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"0966022572\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Spoils-War-Destroyer-Warren-Murphy/dp/0759252343/ref\\u003dsr_1_1/188-0805395-8182724?ie\\u003dUTF8\\u0026keywords\\u003d9780759252349\\u0026qid\\u003d1434508721\\u0026sr\\u003d8-1\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"0759252343\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/gp/product/1619044161/ref\\u003ds9_simh_gw_p63_d3_i4?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-1\\u0026pf_rd_r\\u003d9LG2PTD6QGEZHREY432U\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d534380538\\u0026pf_rd_i\\u003d500944\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1619044161\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/gp/product/1277240469/ref\\u003ds9_ri_gw_g201_i9?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-2\\u0026pf_rd_r\\u003dOKXD96G4LIO1M5DK7256\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d747902558\\u0026pf_rd_i\\u003d884179\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1277240469\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/product/1277760233/ref\\u003ds9_simh_gw_p63_d3_i3?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-2\\u0026pf_rd_r\\u003d192K1XBMKJWZU4U9T670\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d553226685\\u0026pf_rd_i\\u003d556820\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1277760233\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/gp/product/1288354339/ref\\u003ds9_simh_gw_p107_d0_i4?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-3\\u0026pf_rd_r\\u003dSO55W703EIKF63LSATHH\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d576464099\\u0026pf_rd_i\\u003d961684\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1288354339\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/gp/product/B000Y9M2Z8/ref\\u003dsr_1_5?ie\\u003dUTF8\\u0026qid\\u003d68957459\\u0026sr\\u003d8-1\\u0026keywords\\u003dnull\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"B000Y9M2Z8\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/product/1142716082/ref\\u003ds9_ri_gw_g201_i3?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-2\\u0026pf_rd_r\\u003dF73P3Q3Q42E0MA9IR3WJ\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d544034392\\u0026pf_rd_i\\u003d971120\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1142716082\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/product/1436693578/ref\\u003ds9_simh_gw_p107_d0_i7?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-1\\u0026pf_rd_r\\u003d99O6Y6T9A8GS7R5720B0\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d496554063\\u0026pf_rd_i\\u003d824861\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1436693578\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Stepladder-Fiberglass-ft-375-Capacity/dp/B002AZI49M/ref\\u003dsr_1_1/183-1355335-6997219?ie\\u003dUTF8\\u0026keywords\\u003d0094703196900\\u0026qid\\u003d1434659776\\u0026sr\\u003d8-1\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"B002AZI49M\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/3790815721/ref\\u003dsr_1_1?ie\\u003dUTF8\\u0026qid\\u003d47125543\\u0026sr\\u003d8-1\\u0026keywords\\u003dnull\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"3790815721\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/product/1780830041/ref\\u003ds9_simh_gw_p79_d17_i6?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-1\\u0026pf_rd_r\\u003dN36493T45F7KM5DST4XD\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d681355952\\u0026pf_rd_i\\u003d407137\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1780830041\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/product/1492201278/ref\\u003dsr_1_10?ie\\u003dUTF8\\u0026qid\\u003d60008119\\u0026sr\\u003d8-1\\u0026keywords\\u003dnull\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1492201278\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Moral-Meaning-Bible-1/dp/0976933020/179-3617094-7389914?ie\\u003dUTF8\\u0026redirect\\u003dtrue\\u0026ref_\\u003ds9_simh_gw_p63_d3_i4\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"0976933020\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Artist-Garden-Walled-Great-Glemham/dp/0956186998\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"0956186998\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Power-Techniques-Improve-Memory-Procrastination/dp/1419399381/ref\\u003dsr_1_1/187-0087211-5658230?ie\\u003dUTF8\\u0026keywords\\u003d1419399381\\u0026qid\\u003d1434661016\\u0026sr\\u003d8-1\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1419399381\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/0321982991/ref\\u003dcm_cr_pr_product_top\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"0321982991\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/product/1937303098/ref\\u003ds9_simh_gw_p74_d2_i6?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-2\\u0026pf_rd_r\\u003dPE5RJS9RG897M86VBG55\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d287461703\\u0026pf_rd_i\\u003d730469\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1937303098\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/gp/product/1140905902/ref\\u003dsr_1_3?ie\\u003dUTF8\\u0026qid\\u003d12249211\\u0026sr\\u003d8-1\\u0026keywords\\u003dnull\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1140905902\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/1452668159/ref\\u003dcm_cr_pr_product_top\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"1452668159\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/product/0500282234/ref\\u003ds9_simh_gw_p79_d17_i2?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-1\\u0026pf_rd_r\\u003dIR0529FYG68I106DR7SL\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d292698592\\u0026pf_rd_i\\u003d436709\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"0500282234\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Overland-Beyond-Theresa-Hewat/dp/0903909138/ref\\u003dsr_1_1/190-6495656-9434655?ie\\u003dUTF8\\u0026keywords\\u003d0903909138\\u0026qid\\u003d1434660544\\u0026sr\\u003d8-1\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"0903909138\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Vanity-Fair-Coverage-Underwire-75346/dp/B00Q7DN5VU/ref\\u003dsr_1_1/166-8687063-8587826?ie\\u003dUTF8\\u0026keywords\\u003d0083623777443\\u0026qid\\u003d1473951935\\u0026sr\\u003d8-1\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"metaCategId\\\":11450,\\\"leafCategId\\\":63853,\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"B00Q7DN5VU\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Kurt-Adler-Dessert-Recipe-Ornament/dp/B018JQXHPG/ref\\u003dsr_1_2/162-8279833-5061559?ie\\u003dUTF8\\u0026keywords\\u003d0086131332074\\u0026qid\\u003d1473951951\\u0026sr\\u003d8-2\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"metaCategId\\\":1,\\\"leafCategId\\\":148,\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"B018JQXHPG\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Beyond-Geometry-Experiments-Form-1940s-1970s/dp/0262240475\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"0262240475\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/gp/product/162459042X/ref\\u003ds9_ri_gw_g201_i10?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-3\\u0026pf_rd_r\\u003d32MV0S8PFB49KX9B508J\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d645364713\\u0026pf_rd_i\\u003d157500\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"162459042X\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/gp/product/B002OHDRF2/ref\\u003ds9_ri_gw_g201_i10?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-3\\u0026pf_rd_r\\u003dN2S87N6Z398REUVM1QMV\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d456084397\\u0026pf_rd_i\\u003d260036\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"B002OHDRF2\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/ALGEBRA-THIRD-330-H-LANE-BIRKHOFF/dp/B00KIG9HNW/ref\\u003dsr_1_2/175-1006394-2907034?ie\\u003dUTF8\\u0026keywords\\u003d9780821816462\\u0026qid\\u003d1434674736\\u0026sr\\u003d8-2\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"B00KIG9HNW\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/product/0743247612/ref\\u003ds9_simh_gw_p79_d17_i5?pf_rd_m\\u003dATVPDKIKX0DER\\u0026pf_rd_s\\u003dcenter-2\\u0026pf_rd_r\\u003d9QI92D9ZA2PGEU1S67Y8\\u0026pf_rd_t\\u003d101\\u0026pf_rd_p\\u003d491952717\\u0026pf_rd_i\\u003d133029\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"0743247612\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Building-Procurement-Surveying-Alan-Turner/dp/0333522869/ref\\u003dsr_1_1/180-3512075-9002103?ie\\u003dUTF8\\u0026keywords\\u003d9780333522868\\u0026qid\\u003d1434926616\\u0026sr\\u003d8-1\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"0333522869\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Persecution-Prejudice-Integrated-humanities-David/dp/0582025095/ref\\u003dsr_1_1/192-2160439-7754248?ie\\u003dUTF8\\u0026keywords\\u003d9780582025097\\u0026qid\\u003d1434520194\\u0026sr\\u003d8-1\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"0582025095\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/dp/product/B006GHAPYC/ref\\u003dsr_1_7?ie\\u003dUTF8\\u0026qid\\u003d46028791\\u0026sr\\u003d8-1\\u0026keywords\\u003dnull\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"B006GHAPYC\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}","{\"url\":\"https://www.amazon.com/Walt-Disney-His-Life-Pictures/dp/0786831162/ref\\u003dsr_1_1/182-6322434-7758236?ie\\u003dUTF8\\u0026keywords\\u003d9780786831166\\u0026qid\\u003d1434675102\\u0026sr\\u003d8-1\",\"metadata\":\"{\\\"crawlType\\\":\\\"PRODUCT\\\",\\\"retailerName\\\":\\\"amazon_us\\\",\\\"site\\\":\\\"US\\\",\\\"productCrawlMetadata\\\":{\\\"retailerProductId\\\":\\\"0786831162\\\",\\\"referencePrice\\\":0.0,\\\"referenceShippingPrice\\\":0.0,\\\"selectors\\\":[]},\\\"tracker\\\":{\\\"crawlRequestCreateTS\\\":1486899602000}}\",\"linkSelectors\":\"\",\"extrFeilds\":[]}"]} \ No newline at end of file diff --git a/measure/src/test/resources/output.msg b/measure/src/test/resources/output.msg deleted file mode 100644 index 4ec9288b4..000000000 --- a/measure/src/test/resources/output.msg +++ /dev/null @@ -1 +0,0 @@ -{"groups":[{"_name":"DefaultGroup","groupType":"FULLSITE","storeType":"CATALOG","attrsList":[{"name":"CENTERCOL","dataType":"STRING","values":["\u003cdiv id\u003d\"centerCol\" class\u003d\"centerColumn\"\u003e \u003cdiv id\u003d\"booksTitle\" class\u003d\"feature\" data-feature-name\u003d\"booksTitle\"\u003e \u003cdiv class\u003d\"a-section a-spacing-none\"\u003e \u003ch1 id\u003d\"title\" class\u003d\"a-size-large a-spacing-none\"\u003e \u003cspan id\u003d\"productTitle\" class\u003d\"a-size-large\"\u003eBefore the First Day\u003c/span\u003e \u003cspan class\u003d\"a-size-medium a-color-secondary a-text-normal\"\u003ePaperback\u003c/span\u003e \u003c!-- use pre formatted date that complies with legal requirement from media matrix --\u003e \u003cspan class\u003d\"a-size-medium a-color-secondary a-text-normal\"\u003e– July 22, 2013\u003c/span\u003e \u003c/h1\u003e \u003c/div\u003e \u003cdiv id\u003d\"byline\" class\u003d\"a-section a-spacing-micro bylineHidden feature\"\u003e by \u003cspan class\u003d\"author notFaded\" data-width\u003d\"\"\u003e \u003ca class\u003d\"a-link-normal\" href\u003d\"/s/ref\u003ddp_byline_sr_book_1?ie\u003dUTF8\u0026amp;text\u003dStephen+M.+Hale\u0026amp;search-alias\u003dbooks\u0026amp;field-author\u003dStephen+M.+Hale\u0026amp;sort\u003drelevancerank\"\u003eStephen M. Hale\u003c/a\u003e \u003cspan class\u003d\"contribution\" spacing\u003d\"none\"\u003e \u003cspan class\u003d\"a-color-secondary\"\u003e(Author)\u003c/span\u003e \u003c/span\u003e \u003c/span\u003e \u003c/div\u003e \u003c/div\u003e \u003cdiv id\u003d\"averageCustomerReviews_feature_div\" class\u003d\"feature\" data-feature-name\u003d\"averageCustomerReviews\"\u003e \u003cstyle type\u003d\"text/css\"\u003e /* * Fix for UDP-1061. Average customer reviews has a small extra line on hover * https://omni-grok.amazon.com/xref/src/appgroup/websiteTemplates/retail/SoftlinesDetailPageAssets/udp-intl-lock/src/legacy.css?indexName\u003dWebsiteTemplates#40 */ .noUnderline a:hover { text-decoration: none; }\u003c/style\u003e \u003cdiv id\u003d\"averageCustomerReviews\" class\u003d\"a-spacing-none\" data-asin\u003d\"1628391340\" data-ref\u003d\"dpx_acr_pop_\"\u003e \u003cspan class\u003d\"a-declarative\" data-action\u003d\"acrStarsLink-click-metrics\" data-acrstarslink-click-metrics\u003d\"{}\"\u003e \u003cspan id\u003d\"acrPopover\" class\u003d\"reviewCountTextLinkedHistogram noUnderline\" title\u003d\"5.0 out of 5 stars\"\u003e \u003cspan class\u003d\"a-declarative\" data-action\u003d\"a-popover\" data-a-popover\u003d\"{\u0026quot;max-width\u0026quot;:\u0026quot;700\u0026quot;,\u0026quot;closeButton\u0026quot;:\u0026quot;false\u0026quot;,\u0026quot;position\u0026quot;:\u0026quot;triggerBottom\u0026quot;,\u0026quot;url\u0026quot;:\u0026quot;/gp/customer-reviews/widgets/average-customer-review/popover/ref\u003ddpx_acr_pop_?contextId\u003ddpx\u0026amp;asin\u003d1628391340\u0026quot;}\"\u003e \u003ca href\u003d\"javascript:void(0)\" class\u003d\"a-popover-trigger a-declarative\"\u003e \u003ci class\u003d\"a-icon a-icon-star a-star-5\"\u003e\u003cspan class\u003d\"a-icon-alt\"\u003e5.0 out of 5 stars\u003c/span\u003e\u003c/i\u003e \u003ci class\u003d\"a-icon a-icon-popover\"\u003e\u003c/i\u003e\u003c/a\u003e \u003c/span\u003e \u003cspan class\u003d\"a-letter-space\"\u003e\u003c/span\u003e \u003c/span\u003e \u003c/span\u003e \u003cspan class\u003d\"a-letter-space\"\u003e\u003c/span\u003e \u003cspan class\u003d\"a-declarative\" data-action\u003d\"acrLink-click-metrics\" data-acrlink-click-metrics\u003d\"{}\"\u003e \u003ca id\u003d\"acrCustomerReviewLink\" class\u003d\"a-link-normal\" href\u003d\"#customerReviews\"\u003e \u003cspan id\u003d\"acrCustomerReviewText\" class\u003d\"a-size-base\"\u003e1 customer review\u003c/span\u003e \u003c/a\u003e \u003c/span\u003e \u003cscript type\u003d\"text/javascript\"\u003e P.when(\u0027A\u0027, \u0027ready\u0027).execute(function(A) { A.declarative(\u0027acrLink-click-metrics\u0027, \u0027click\u0027, { \"allowLinkDefault\" : true }, function(event){ if(window.ue) { ue.count(\"acrLinkClickCount\", (ue.count(\"acrLinkClickCount\") || 0) + 1); } }); }); \u003c/script\u003e \u003cscript type\u003d\"text/javascript\"\u003e P.when(\u0027A\u0027, \u0027cf\u0027).execute(function(A) { A.declarative(\u0027acrStarsLink-click-metrics\u0027, \u0027click\u0027, { \"allowLinkDefault\" : true }, function(event){ if(window.ue) { ue.count(\"acrStarsLinkWithPopoverClickCount\", (ue.count(\"acrStarsLinkWithPopoverClickCount\") || 0) + 1); } }); }); \u003c/script\u003e \u003c/div\u003e \u003c/div\u003e \u003cdiv id\u003d\"seriesTitle_feature_div\" class\u003d\"feature\" data-feature-name\u003d\"seriesTitle\"\u003e \u003c/div\u003e \u003cdiv id\u003d\"zeitgeistBadge_feature_div\" class\u003d\"feature\" data-feature-name\u003d\"zeitgeistBadge\"\u003e \u003c/div\u003e \u003cdiv id\u003d\"socialFabric_feature_div\" class\u003d\"feature\" data-feature-name\u003d\"socialFabric\"\u003e \u003c/div\u003e \u003chr /\u003e \u003cdiv id\u003d\"adoptedData\" class\u003d\"feature\" data-feature-name\u003d\"adoptedData\"\u003e \u003cstyle\u003e #adoptedDataContainer .campusLogo { height:20px; width:20px; margin-right:1px; } #adoptedDataContainer .campusName { line-height:20px; } #adoptedDataContainer.paddingMicro { padding:4px !important; } #adoptedDataContainer .paddingLeftMicro { padding-left:4px !important; } #adoptedDataContainer .paddingLeftSmall { padding-left:10px !important; } #adoptedDataContainer .paddingRightSmall { padding-right:10px !important; } #adoptedDataContainer .courseListExpanderContainer a:focus { outline:none; } #adoptedDataContainer .courseColumn { width:50%; float:left; } #adoptedDataContainer .textRow { display:inline-block; width:100%; }\u003c/style\u003e \u003c/div\u003e \u003cdiv id\u003d\"MediaMatrix\" class\u003d\"feature\" data-feature-name\u003d\"MediaMatrix\"\u003e \u003cdiv id\u003d\"formats\" class\u003d\"a-section a-spacing-large responsive\"\u003e \u003ca class\u003d\"a-link-expander a-spacing-top-micro a-spacing-small a-size-small\"\u003e \u003ci id\u003d\"formatsIcon\" class\u003d\"a-icon a-icon-expand\"\u003e\u003c/i\u003e \u003cspan id\u003d\"showMoreFormatsPrompt\" class\u003d\"tmmShowPrompt\"\u003eSee all 3 formats and editions\u003c/span\u003e \u003cspan id\u003d\"hideMoreFormatsPrompt\" class\u003d\"tmmHidePrompt\"\u003eHide other formats and editions\u003c/span\u003e \u003c/a\u003e \u003cdiv id\u003d\"twister\" class\u003d\"a-section a-spacing-base a-spacing-top-small a-size-mini\"\u003e \u003c!-- This file is just a replica of table-aiv present in mediaMatrixtemplate package. This has just been picked up and moved and there has not been addition/modification/removal of any logic here . --\u003e \u003ctable class\u003d\"a-normal a-spacing-none title\"\u003e \u003ctbody\u003e \u003ctr\u003e \u003ctd class\u003d\"dp-title-col\"\u003e \u003c/td\u003e \u003ctd class\u003d\"a-color-secondary a-text-right dp-price-col\"\u003e \u003cdiv class\u003d\"a-fixed-right-grid\"\u003e \u003cdiv class\u003d\"a-fixed-right-grid-inner\" style\u003d\"padding-right:50px\"\u003e \u003cdiv class\u003d\"a-fixed-right-grid-col a-col-left\" style\u003d\"padding-right:2.5%;*width:97.1%;float:left;\"\u003e Price \u003c/div\u003e \u003cdiv class\u003d\"a-fixed-right-grid-col a-col-right\" style\u003d\"width:50px;margin-right:-50px;float:left;\"\u003e\u003c/div\u003e \u003c/div\u003e \u003c/div\u003e \u003c/td\u003e \u003ctd class\u003d\"a-color-secondary a-text-right dp-new-col\"\u003e New from \u003c/td\u003e \u003ctd class\u003d\"a-color-secondary a-text-right dp-used-col\"\u003e Used from \u003c/td\u003e \u003c/tr\u003e \u003c/tbody\u003e \u003c/table\u003e \u003cdiv class\u003d\"hr unselected-row\"\u003e\u003c/div\u003e \u003cdiv class\u003d\"top-level unselected-row\"\u003e \u003cspan class\u003d\"a-declarative\" data-action\u003d\"tmm-see-more-editions-click\" data-tmm-see-more-editions-click\u003d\"{\u0026quot;metabindingPlaceHolder\u0026quot;:1,\u0026quot;metabindingPlaceHolderState\u0026quot;:\u0026quot;0\u0026quot;,\u0026quot;metabindingUrl\u0026quot;:\u0026quot;/Before-First-Day-Stephen-Hale-ebook/dp/B00E4WYJT8/ref\u003dtmm_kin_title_0?_encoding\u003dUTF8\u0026amp;amp;qid\u003d\u0026amp;amp;sr\u003d\u0026quot;}\" id\u003d\"declarative_1\"\u003e \u003ctable class\u003d\"a-normal a-spacing-none\"\u003e \u003ctbody\u003e \u003ctr\u003e \u003ctd class\u003d\"dp-title-col\"\u003e \u003c!-- Do not show expander when DVD_redesign weblab is in treatment --\u003e \u003ca class\u003d\"empty-expander title-expander\" style\u003d\"vertical-align:middle;\"\u003e \u003cspan class\u003d\"metabinding-expander\"\u003e\u003c/span\u003e \u003c/a\u003e \u003ca href\u003d\"/Before-First-Day-Stephen-Hale-ebook/dp/B00E4WYJT8/ref\u003dtmm_kin_title_0?_encoding\u003dUTF8\u0026amp;qid\u003d\u0026amp;sr\u003d\" class\u003d\"title-text\"\u003e \u003cspan class\u003d\"a-size-small a-color-base\"\u003eKindle\u003c/span\u003e \u003cspan id\u003d\"tmmSpinnerDiv_1\" style\u003d\"display: none\" class\u003d\"tmmAjaxLoading\"\u003e\u003c/span\u003e \u003c/a\u003e \u003cspan id\u003d\"tmmErrorDiv_1\" class\u003d\"tmmErrorClass displayNone\"\u003e \u003cdiv class\u003d\"a-box a-alert-inline a-alert-inline-error\"\u003e \u003cdiv class\u003d\"a-box-inner a-alert-container\"\u003e \u003ci class\u003d\"a-icon a-icon-alert\"\u003e\u003c/i\u003e \u003cdiv class\u003d\"a-alert-content\"\u003e \u003cspan\u003e\u0026quot;Please retry\u0026quot;\u003c/span\u003e \u003c/div\u003e \u003c/div\u003e \u003c/div\u003e \u003c/span\u003e \u003c/td\u003e \u003ctd class\u003d\"a-text-right dp-price-col\"\u003e \u003cdiv class\u003d\"a-fixed-right-grid\"\u003e \u003cdiv class\u003d\"a-fixed-right-grid-inner\" style\u003d\"padding-right:50px\"\u003e \u003cdiv class\u003d\"a-fixed-right-grid-col a-col-left\" style\u003d\"padding-right:2.5%;*width:97.1%;float:left;\"\u003e \u003ca class\u003d\"a-link-normal\" href\u003d\"/Before-First-Day-Stephen-Hale-ebook/dp/B00E4WYJT8/ref\u003dtmm_kin_title_0?_encoding\u003dUTF8\u0026amp;qid\u003d\u0026amp;sr\u003d\"\u003e \u003cspan class\u003d\"a-size-small a-color-price\"\u003e$7.99\u003c/span\u003e \u003c/a\u003e \u003c/div\u003e \u003cdiv class\u003d\"a-fixed-right-grid-col a-col-right\" style\u003d\"width:50px;margin-right:-50px;float:left;\"\u003e \u003c/div\u003e \u003c/div\u003e \u003c/div\u003e \u003c/td\u003e \u003ctd class\u003d\"a-color-tertiary a-text-right dp-new-col\"\u003e — \u003c/td\u003e \u003ctd class\u003d\"a-color-tertiary a-text-right dp-used-col\"\u003e — \u003c/td\u003e \u003c/tr\u003e \u003c/tbody\u003e \u003c/table\u003e \u003c/span\u003e \u003c/div\u003e \u003cdiv id\u003d\"metabinding_row_top_1\" class\u003d\"second-level\"\u003e\u003c/div\u003e \u003cdiv id\u003d\"metabinding_row_bottom_1\" class\u003d\"second-level\"\u003e\u003c/div\u003e \u003cdiv class\u003d\"hr selected-row\"\u003e\u003c/div\u003e \u003cdiv class\u003d\"top-level selected-row\"\u003e \u003cspan class\u003d\"a-declarative\" data-action\u003d\"tmm-see-more-editions-click\" data-tmm-see-more-editions-click\u003d\"{\u0026quot;metabindingPlaceHolder\u0026quot;:2,\u0026quot;metabindingPlaceHolderState\u0026quot;:\u0026quot;0\u0026quot;,\u0026quot;metabindingUrl\u0026quot;:\u0026quot;#\u0026quot;}\" id\u003d\"declarative_2\"\u003e \u003ctable class\u003d\"a-normal a-spacing-none\"\u003e \u003ctbody\u003e \u003ctr\u003e \u003ctd class\u003d\"dp-title-col\"\u003e \u003c!-- Do not show expander when DVD_redesign weblab is in treatment --\u003e \u003ca class\u003d\"a-link-expander title-expander\" style\u003d\"vertical-align:middle; padding-left:16px\"\u003e \u003cspan class\u003d\"a-link-expander metabinding-expander\"\u003e \u003ci id\u003d\"editionsIcon_2\" class\u003d\"a-icon a-icon-expand tmm-row-icon\"\u003e\u003c/i\u003e \u003c/span\u003e \u003c/a\u003e \u003cspan class\u003d\"title-text\"\u003e \u003cspan class\u003d\"a-size-small a-color-base\"\u003ePaperback\u003c/span\u003e \u003cspan id\u003d\"tmmSpinnerDiv_2\" style\u003d\"display: none\" class\u003d\"tmmAjaxLoading\"\u003e\u003c/span\u003e \u003c/span\u003e \u003cspan id\u003d\"tmmErrorDiv_2\" class\u003d\"tmmErrorClass displayNone\"\u003e \u003cdiv class\u003d\"a-box a-alert-inline a-alert-inline-error\"\u003e \u003cdiv class\u003d\"a-box-inner a-alert-container\"\u003e \u003ci class\u003d\"a-icon a-icon-alert\"\u003e\u003c/i\u003e \u003cdiv class\u003d\"a-alert-content\"\u003e \u003cspan\u003e\u0026quot;Please retry\u0026quot;\u003c/span\u003e \u003c/div\u003e \u003c/div\u003e \u003c/div\u003e \u003c/span\u003e \u003c/td\u003e \u003ctd class\u003d\"a-text-right dp-price-col\"\u003e \u003cdiv class\u003d\"a-fixed-right-grid\"\u003e \u003cdiv class\u003d\"a-fixed-right-grid-inner\" style\u003d\"padding-right:50px\"\u003e \u003cdiv class\u003d\"a-fixed-right-grid-col a-col-left\" style\u003d\"padding-right:2.5%;*width:97.1%;float:left;\"\u003e \u003cspan class\u003d\"a-size-small a-color-price\"\u003e$15.49\u003c/span\u003e \u003c/div\u003e \u003cdiv class\u003d\"a-fixed-right-grid-col a-col-right\" style\u003d\"width:50px;margin-right:-50px;float:left;\"\u003e \u003c/div\u003e \u003c/div\u003e \u003c/div\u003e \u003c/td\u003e \u003ctd class\u003d\"a-text-right dp-new-col\"\u003e \u003ca class\u003d\"a-link-normal\" href\u003d\"/gp/offer-listing/1628391340/ref\u003dtmm_pap_new_olp_sr?ie\u003dUTF8\u0026amp;condition\u003dnew\u0026amp;qid\u003d\u0026amp;sr\u003d\"\u003e \u003cspan\u003e$9.44\u003c/span\u003e \u003c/a\u003e \u003c/td\u003e \u003ctd class\u003d\"a-text-right dp-used-col\"\u003e \u003ca class\u003d\"a-link-normal\" href\u003d\"/gp/offer-listing/1628391340/ref\u003dtmm_pap_used_olp_sr?ie\u003dUTF8\u0026amp;condition\u003dused\u0026amp;qid\u003d\u0026amp;sr\u003d\"\u003e \u003cspan\u003e$12.17\u003c/span\u003e \u003c/a\u003e \u003c/td\u003e \u003c/tr\u003e \u003c/tbody\u003e \u003c/table\u003e \u003c/span\u003e \u003c/div\u003e \u003cdiv id\u003d\"metabinding_row_top_2\" class\u003d\"second-level\"\u003e\u003c/div\u003e \u003cdiv id\u003d\"metabinding_row_bottom_2\" class\u003d\"second-level\"\u003e\u003c/div\u003e \u003cscript type\u003d\"a-state\" data-a-state\u003d\"{\u0026quot;key\u0026quot;:\u0026quot;mediamatrix-state\u0026quot;}\"\u003e{\"url_2\":\"/gp/media-matrix/fetch-DPX-expansion-data.html/ref\u003dtmm_pap_metabinding_expansion_sr?ie\u003dUTF8\u0026amp;tagActionCode\u003d\u0026amp;bindingCount\u003d2\u0026amp;websiteDisplayGroup\u003dbook_display_on_website\u0026amp;cor\u003dUS\u0026amp;storeID\u003d\u0026amp;qid\u003d\u0026amp;productTypeDefinition\u003dABIS_BOOK\u0026amp;viewID\u003dglance\u0026amp;landingAsin\u003d1628391340\u0026amp;customerID\u003d\u0026amp;tasParentAsin\u003dB00E5D53IM\u0026amp;asin\u003d1628391340\u0026amp;metabinding\u003dpaperback_meta_binding\u0026amp;metabindingIndex\u003d2\u0026amp;isPrime\u003d0\u0026amp;nodeID\u003d\u0026amp;sr\u003d\",\"isAjaxInProgress_1\":\"0\",\"url_1\":\"/gp/media-matrix/fetch-DPX-expansion-data.html/ref\u003dtmm_kin_metabinding_expansion_0?ie\u003dUTF8\u0026amp;tagActionCode\u003d\u0026amp;bindingCount\u003d1\u0026amp;websiteDisplayGroup\u003dbook_display_on_website\u0026amp;cor\u003dUS\u0026amp;storeID\u003d\u0026amp;qid\u003d\u0026amp;productTypeDefinition\u003dABIS_BOOK\u0026amp;viewID\u003dglance\u0026amp;landingAsin\u003d1628391340\u0026amp;customerID\u003d\u0026amp;tasParentAsin\u003dB00E5D53IM\u0026amp;asin\u003dB00E4WYJT8\u0026amp;metabinding\u003dkindle_meta_binding\u0026amp;metabindingIndex\u003d1\u0026amp;isPrime\u003d0\u0026amp;nodeID\u003d\u0026amp;sr\u003d\",\"isAjaxInProgress_2\":\"0\",\"isAjaxComplete_1\":\"0\",\"isAjaxComplete_2\":\"0\"}\u003c/script\u003e \u003c/div\u003e \u003cdiv id\u003d\"tmmSwatches\" class\u003d\"a-row nonJSFormats\"\u003e \u003cul class\u003d\"a-nostyle a-button-list a-horizontal\"\u003e \u003cli class\u003d\"swatchElement unselected\"\u003e\u003cspan class\u003d\"a-list-item\"\u003e \u003cspan class\u003d\"a-button a-spacing-mini a-button-toggle format\"\u003e\u003cspan class\u003d\"a-button-inner\"\u003e\u003ca href\u003d\"/Before-First-Day-Stephen-Hale-ebook/dp/B00E4WYJT8/ref\u003dtmm_kin_swatch_0?_encoding\u003dUTF8\u0026amp;qid\u003d\u0026amp;sr\u003d\" class\u003d\"a-button-text\" role\u003d\"button\"\u003e \u003cspan\u003eKindle\u003c/span\u003e \u003cbr /\u003e \u003cspan class\u003d\"a-color-secondary\"\u003e \u003cspan\u003e $7.99 \u003c/span\u003e \u003c/span\u003e \u003c/a\u003e\u003c/span\u003e\u003c/span\u003e \u003cspan class\u003d\"tmm-olp-links\"\u003e \u003c/span\u003e \u003cspan class\u003d\"a-size-mini a-color-secondary tmm-olp-links\"\u003e \u003ca id\u003d\"kcpAppsPopOver\" class\u003d\"a-size-mini a-link-normal kcpAppsPopOver\" href\u003d\"javascript:void(0);\"\u003e \u003cspan class\u003d\"kcpAppBaseBox_\"\u003e \u003cspan class\u003d\"kcpAppsPopOver\"\u003e Read with Our \u003cspan class\u003d\"a-color-price a-text-bold\"\u003e Free App \u003c/span\u003e \u003c/span\u003e \u003c/span\u003e \u003c/a\u003e \u003c/span\u003e \u003cspan class\u003d\"tmm-olp-links\"\u003e \u003c/span\u003e \u003c/span\u003e\u003c/li\u003e \u003cli class\u003d\"swatchElement selected\"\u003e\u003cspan class\u003d\"a-list-item\"\u003e \u003cspan class\u003d\"a-button a-button-selected a-spacing-mini a-button-toggle format\"\u003e\u003cspan class\u003d\"a-button-inner\"\u003e\u003ca href\u003d\"javascript:void(0)\" class\u003d\"a-button-text\" role\u003d\"button\"\u003e \u003cspan\u003ePaperback\u003c/span\u003e \u003cbr /\u003e \u003cspan class\u003d\"a-color-base\"\u003e \u003cspan class\u003d\"a-color-price\"\u003e $15.49 \u003c/span\u003e \u003c/span\u003e \u003c/a\u003e\u003c/span\u003e\u003c/span\u003e \u003cspan class\u003d\"tmm-olp-links\"\u003e \u003c/span\u003e \u003cspan class\u003d\"tmm-olp-links\"\u003e \u003cspan class\u003d\"olp-used olp-link\"\u003e \u003ca class\u003d\"a-size-mini a-link-normal\" href\u003d\"/gp/offer-listing/1628391340/ref\u003dtmm_pap_used_olp_sr?ie\u003dUTF8\u0026amp;condition\u003dused\u0026amp;qid\u003d\u0026amp;sr\u003d\"\u003e 2 Used \u003cspan class\u003d\"olp-from\"\u003efrom\u003c/span\u003e $12.17 \u003c/a\u003e \u003c/span\u003e \u003cspan class\u003d\"olp-new olp-link\"\u003e \u003ca class\u003d\"a-size-mini a-link-normal\" href\u003d\"/gp/offer-listing/1628391340/ref\u003dtmm_pap_new_olp_sr?ie\u003dUTF8\u0026amp;condition\u003dnew\u0026amp;qid\u003d\u0026amp;sr\u003d\"\u003e 13 New \u003cspan class\u003d\"olp-from\"\u003efrom\u003c/span\u003e $9.44 \u003c/a\u003e \u003c/span\u003e \u003c/span\u003e \u003c/span\u003e\u003c/li\u003e \u003c/ul\u003e \u003c/div\u003e \u003c/div\u003e \u003c/div\u003e \u003cdiv id\u003d\"applicablePromotionList_feature_div\" class\u003d\"feature\" data-feature-name\u003d\"applicablePromotionList\"\u003e \u003c/div\u003e \u003cdiv id\u003d\"holidayDeliveryMessage_feature_div\" class\u003d\"feature\" data-feature-name\u003d\"holidayDeliveryMessage\"\u003e \u003c/div\u003e \u003cdiv id\u003d\"campusInfo\" class\u003d\"feature\" data-feature-name\u003d\"campusInfo\"\u003e \u003c/div\u003e \u003cdiv id\u003d\"extraProductInfoFeatureGroup\" class\u003d\"feature\" data-feature-name\u003d\"extraProductInfoFeatureGroup\"\u003e \u003cdiv id\u003d\"newerVersion_feature_div\" class\u003d\"feature\" data-feature-name\u003d\"newerVersion\"\u003e \u003c/div\u003e \u003cdiv id\u003d\"productAlert_feature_div\" class\u003d\"feature\" data-feature-name\u003d\"productAlert\"\u003e \u003c/div\u003e \u003c/div\u003e \u003cdiv id\u003d\"dynamicIframe_feature_div\" class\u003d\"feature\" data-feature-name\u003d\"dynamicIframe\"\u003e \u003cscript id\u003d\"dynamic_iframe_CSS\" type\u003d\"text/undefined\"\u003ehtml { font-size: 100%;}a:focus { outline: thin dotted #333333; outline-offset: -2px;}a:hover, a:active { outline: 0 none;}body { background-color: #FFFFFF; margin:0; /* from AUI */ color: #333333; font-family: Arial,sans-serif; font-size: 13px; line-height: 19px;}h1, h2, h3, h4, h5, h6 { margin: 0; padding: 0;}h1, h2, h3, h4 { font-family: Arial,sans-serif; text-rendering: optimizelegibility; padding-bottom: 4px;}h1:last-child, h2:last-child, h3:last-child, h4:last-child { padding-bottom: 0;}h1 { font-size: 28px; font-weight: normal; line-height: 1.2;}h2 { font-size: 21px; font-weight: normal; line-height: 1.3;}h3 { font-size: 17px; font-weight: normal; line-height: 1.24;}p { margin: 0 0 14px; padding: 0;}p:last-child { margin-bottom: 0;}p + p { margin-top: -4px;}b, strong { font-weight: bold;}i, em { font-style: italic;}blockquote { margin: 13px;}small { font-size: 12px;}img { border: 0 none; max-width: 100%; vertical-align: top;}td { float: none; margin-right: 0;}td, th { padding: 3px; vertical-align: top;}td:first-child, th:first-child { padding-left: 0;}td:last-child, th:last-child { padding-right: 0;}th { text-align: left;}tr:last-child td, tr:last-child th { padding-bottom: 0;}tr:first-child td, tr:first-child th { padding-top: 0;}a, a:visited, a:active, a:link { color: #007EB9; text-decoration: none;}a:hover { color: #E47911; cursor: pointer; text-decoration: underline;}p a { text-decoration: underline;}ul { /*color: #AAAAAA; Overriding this style as the color is out of sync with the rest of book description*/ list-style-type: disc;}ol { color: #888888;}ul, ol { margin: 0 0 18px 22px; padding: 0;}ul li, ol li { margin: 0; word-wrap: break-word;}ul:last-child, ol:last-child { margin-bottom: 0 !important;}#iframeContent {overflow: hidden;}h2.productDescriptionHeader { margin-bottom: 0em;}.emptyClear { clear:left; height:0px; font-size:0px;}div.productDescriptionWrapper { margin: 0 0 1em 0;}h3.productDescriptionSource { font-weight:normal; color:#333333; font-size:1.23em; margin: .75em 0 .375em 0; clear:left;}.seeAll { margin-top: 1.25em; margin-left: -15px;}#technicalProductFeatures ul { list-style-type: disc; margin: 1.12em 0; margin-left: 20px;}#iframeContent ul { list-style-type: disc; margin-left: 20px;}ul li { margin: 0 0 0 20px;}ul li ul { list-style-type: disc; margin-left: 20px;}ul li ul li { margin: 0 0 0 20px;}.aplus h4, .aplus h5 { margin: 0 0 .75em 0; font-size: 1em;}.aplus h4 { color: #CC6600;}.aplus p { margin: 0 0 1em 0;}.aplus .break { clear:both; height:0px; font-size:0px;}.aplus .spacer { margin-bottom: 13px;}.aplus img { border:none;}.aplus .leftImage, .aplus .rightImage, .aplus .centerImage { margin-bottom: 1em; margin-top: 0; text-align:center; vertical-align:top;}.aplus .leftImage { margin-right: 15px; float:left; clear:left;}.aplus .rightImage { margin-left: 15px; float:right; clear:right;}.aplus .imageCaption { clear:both; padding: .5em .5em 0 .5em; font-size: .846em; display: block;}.aplus table.data { border-collapse: collapse; margin-bottom: 1.25em;}.aplus table.data th { font-weight: bold; background: #F7F7F7; border-style:solid; border-color: #CCCCCC; border-width:0 0 1px 1px;}.aplus table.data td { border-left: 1px solid #CCC; border-bottom: 1px dotted #CCC}.aplus table.data th, .aplus table.data td{ padding:3px 10px; text-align:left}.aplus table.data tfoot { font-style: italic;}.aplus table.data caption { background: #eee; font-size: .8125em;}.aplus table.data tr td:first-child, .aplus table.data tr th:first-child { border-left-width:0px;}.aplus ul { margin:0 0 1em 0;}.aplus .center { text-align: center;}.aplus .right { text-align: right;}.aplus .sixth-col,.aplus .fourth-col,.aplus .third-col,.aplus .half-col,.aplus .two-third-col,.aplus .three-fourth-col,.aplus .one-col { float:left; margin-right: 1.6760%; overflow: hidden;}.aplus .last { margin-right:0px;}.aplus .sixth-col { width: 15.080%;}.aplus .fourth-col { width: 23.4637%;}.aplus .third-col { width: 31.8436%;}.aplus .half-col { width: 48.6034%;}.aplus .two-third-col { width: 65.3631%;}.aplus .three-fourth-col { width: 73.7430%;}.aplus .one-col { width: 98.8827%; margin-right:0;}.aplus .last { margin-right:0;}.aplus { width: 100%; min-width: 895px;}\u003c/script\u003e \u003cscript type\u003d\"text/javascript\"\u003eP.register(\"DynamicIframe\", function(){function DynamicIframe(options) { var nTries \u003d 0, MAX_TRIES \u003d 20, iframeId \u003d options.iframeId, encodedIframeContent \u003d options.encodedIframeContent, iframeWrapperId \u003d options.iframeWrapperId, initialResizeCallback \u003d options.initialResizeCallback, iframeCSSId \u003d \"dynamic_iframe_CSS\"; iframeOverriddenCSSId \u003d options.overriddenCSSId; this.createIframe \u003d function() { var iframe \u003d document.createElement(\u0027iframe\u0027); iframe.id \u003d iframeId; iframe.className \u003d \"ap_never_hide\"; iframe.width \u003d \"100%\"; iframe.scrolling \u003d \"no\"; iframe.frameBorder \u003d \"0\"; onloadFn \u003d this.iframeload; if ( iframe.addEventListener ) iframe.addEventListener(\"load\", onloadFn, false); else if ( iframe.attachEvent ) iframe.attachEvent(\"onload\", onloadFn); else iframe.onload \u003d onloadFn; var wrapper \u003d document.getElementById(iframeWrapperId); wrapper.innerHTML \u003d \u0027\u0027; iframe \u003d wrapper.appendChild(iframe); var overriddenCSS \u003d \"\"; if (document.getElementById(iframeOverriddenCSSId)) { \toverriddenCSS \u003d document.getElementById(iframeOverriddenCSSId).innerHTML; } var doc \u003d ((iframe.contentWindow \u0026\u0026 iframe.contentWindow.document) || iframe.contentDocument); if (doc \u0026\u0026 doc.open) { doc.open(); doc.writeln(\u0027\u003chtml\u003e\u003chead\u003e\u003cbase target\u003d\"_top\" /\u003e\u003c/\u0027 + \u0027head\u003e\u003cbody\u003e\u003cstyle class\u003d\"text/css\"\u003e\u0027 + document.getElementById(iframeCSSId).innerHTML + overriddenCSS + \u0027\u003c/style\u003e\u003cdiv id\u003d\"iframeContent\"\u003e\u0027 + \u0027\u0027 + decodeURIComponent(encodedIframeContent) + \u0027\u003c/\u0027+\u0027div\u003e\u0027+\u0027\u003c/\u0027+\u0027body\u003e\u003c/html\u003e\u0027); doc.close(); } } this.iframeload \u003d function () { var iframe \u003d document.getElementById(iframeId); iframe.style.display \u003d \u0027\u0027; setTimeout(function () { setIframeHeight(initialResizeCallback); }, 20); } function getDocHeight(doc) { var contentDiv \u003d doc.getElementById(\"iframeContent\"); var docHeight \u003d 0; if(contentDiv){ docHeight \u003d Math.max( contentDiv.scrollHeight, contentDiv.offsetHeight, contentDiv.clientHeight ); } return docHeight; } function setIframeHeight(resizeCallback) { var iframeDoc, iframe \u003d document.getElementById(iframeId); iframeDoc \u003d ((iframe.contentWindow \u0026\u0026 iframe.contentWindow.document) || iframe.contentDocument); if (iframeDoc) { var h \u003d getDocHeight(iframeDoc); if (h \u0026\u0026 h !\u003d 0) { iframe.style.height \u003d parseInt(h) + \u0027px\u0027; if(typeof resizeCallback \u003d\u003d \"function\") { resizeCallback(iframeId);\t } } else if (nTries \u003c MAX_TRIES) { nTries++; setTimeout(function () { setIframeHeight(resizeCallback); }, 50); } } } this.resizeIframe \u003d function(resizeCallback) { nTries \u003d 0;\tsetIframeHeight(resizeCallback); }}return DynamicIframe;});\u003c/script\u003e \u003c/div\u003e \u003cdiv id\u003d\"bookDescription_feature_div\" class\u003d\"feature\" data-feature-name\u003d\"bookDescription\"\u003e \u003cscript id\u003d\"bookDesc_override_CSS\" type\u003d\"text/undefined\"\u003ebody { font-size: 14px; line-height: 1.6em;}.aplus {\tmin-width: inherit;}\u003c/script\u003e \u003cnoscript\u003e \u003cdiv\u003e This is book one of a fantasy series for youths through adults. It is about Narn, a young man in a village of people that is still recent after the Creation. The Maker has Given him the special ability to sense people\u0027s feelings and has Called him to help people when their feelings would turn them from the Maker\u0027s Teachings. Unbeknownst to him, Narn is being pursued by the Blackness, a force from the unMaker trying to undo all that the Maker has Made and that killed Narn\u0027s Father, who also shared Narn\u0027s special sensitivity. Narn discovers that he must choose between the Lessons his Family learned from the Maker and those of the Hunters, seeded from the unMaker. After his best friend, a Hunter, is killed while on a Hunt, Narn\u0027s own emotions blind him, and he ignores his Lessons to avenge his friend\u0027s death. He then finds that he must choose between life and death, not only for the once-revered beast that killed his friend, but also for himself-because his heart has Writings upon it that conflict with what he has seen take place before him. Which is more real? To which will he yield? This book sets the stage for the following series about Narn, which gradually deepens in the question of \u0026quot;choosing whom you will follow,\u0026quot; and gives the reader insight into the Writings on his/her own heart so he/she may be better prepared to make that choice. I have a master\u0027s degree in counseling psychology with special emphasis in personal and family challenges. I began to see that the underlying causes of most people\u0027s problems stemmed from a departure from Standards in their lives, consciously or unconsciously. This realization influenced me very strongly and I found myself writing about it in story form. \u003c/div\u003e \u003cem\u003e\u003c/em\u003e \u003c/noscript\u003e \u003cdiv id\u003d\"outer_postBodyPS\" style\u003d\"overflow: hidden; z-index: 1; height: 0px; display: block;\"\u003e \u003cdiv id\u003d\"postBodyPS\" style\u003d\"overflow: hidden;\"\u003e \u003cdiv id\u003d\"bookDesc_iframe_wrapper\" class\u003d\"maxReadableWidth\"\u003e\u003c/div\u003e \u003c/div\u003e \u003c/div\u003e \u003cdiv id\u003d\"psPlaceHolder\" style\u003d\"height: 20px; display: none;\"\u003e \u003cdiv style\u003d\"z-index: 3;\"\u003e \u003ca class\u003d\"a-link-expander h2-expander\"\u003e \u003ci id\u003d\"bdExpanderIcon\" class\u003d\"a-icon a-icon-expand\"\u003e\u003c/i\u003e \u003cspan id\u003d\"bdSeeAllPrompt\"\u003eRead more\u003c/span\u003e \u003cspan id\u003d\"bdSeeLessPrompt\" style\u003d\"display: none;\"\u003eRead less\u003c/span\u003e \u003c/a\u003e \u003c/div\u003e \u003c/div\u003e \u003cscript type\u003d\"text/javascript\"\u003e P.when(\u0027DynamicIframe\u0027).execute(function(DynamicIframe){\tvar BookDescriptionIframe \u003d null,\t bookDescEncodedData \u003d \"This%20is%20book%20one%20of%20a%20fantasy%20series%20for%20youths%20through%20adults.%20It%20is%20about%20Narn%2C%20a%20young%20man%20in%20a%20village%20of%20people%20that%20is%20still%20recent%20after%20the%20Creation.%20The%20Maker%20has%20Given%20him%20the%20special%20ability%20to%20sense%20people%27s%20feelings%20and%20has%20Called%20him%20to%20help%20people%20when%20their%20feelings%20would%20turn%20them%20from%20the%20Maker%27s%20Teachings.%20Unbeknownst%20to%20him%2C%20Narn%20is%20being%20pursued%20by%20the%20Blackness%2C%20a%20force%20from%20the%20unMaker%20trying%20to%20undo%20all%20that%20the%20Maker%20has%20Made%20and%20that%20killed%20Narn%27s%20Father%2C%20who%20also%20shared%20Narn%27s%20special%20sensitivity.%20Narn%20discovers%20that%20he%20must%20choose%20between%20the%20Lessons%20his%20Family%20learned%20from%20the%20Maker%20and%20those%20of%20the%20Hunters%2C%20seeded%20from%20the%20unMaker.%20After%20his%20best%20friend%2C%20a%20Hunter%2C%20is%20killed%20while%20on%20a%20Hunt%2C%20Narn%27s%20own%20emotions%20blind%20him%2C%20and%20he%20ignores%20his%20Lessons%20to%20avenge%20his%20friend%27s%20death.%20He%20then%20finds%20that%20he%20must%20choose%20between%20life%20and%20death%2C%20not%20only%20for%20the%20once-revered%20beast%20that%20killed%20his%20friend%2C%20but%20also%20for%20himself-because%20his%20heart%20has%20Writings%20upon%20it%20that%20conflict%20with%20what%20he%20has%20seen%20take%20place%20before%20him.%20Which%20is%20more%20real%3F%20To%20which%20will%20he%20yield%3F%20This%20book%20sets%20the%20stage%20for%20the%20following%20series%20about%20Narn%2C%20which%20gradually%20deepens%20in%20the%20question%20of%20%22choosing%20whom%20you%20will%20follow%2C%22%20and%20gives%20the%20reader%20insight%20into%20the%20Writings%20on%20his%2Fher%20own%20heart%20so%20he%2Fshe%20may%20be%20better%20prepared%20to%20make%20that%20choice.%20I%20have%20a%20master%27s%20degree%20in%20counseling%20psychology%20with%20special%20emphasis%20in%20personal%20and%20family%20challenges.%20I%20began%20to%20see%20that%20the%20underlying%20causes%20of%20most%20people%27s%20problems%20stemmed%20from%20a%20departure%20from%20Standards%20in%20their%20lives%2C%20consciously%20or%20unconsciously.%20This%20realization%20influenced%20me%20very%20strongly%20and%20I%20found%20myself%20writing%20about%20it%20in%20story%20form.\",\t bookDescriptionAvailableHeight,\t minBookDescriptionInitialHeight \u003d 112,\t options \u003d {},\t iframeId \u003d \"bookDesc_iframe\"; function resizeCallback() { P.guardFatal(\"bookDescription\", function() { // Get the line-height of the iframe var iframe \u003d document.getElementById(iframeId); var iframeDocument \u003d iframe.contentDocument; if (false \u0026\u0026 iframeDocument \u0026\u0026 iframeDocument.defaultView) { // Set the height to the number of lines specified var numLines \u003d parseInt(0, 10); // Get the line-height of the iframe var iframeContent \u003d iframeDocument.getElementById(\"iframeContent\"); // Compute the line height var lineHeight \u003d iframeDocument.defaultView.getComputedStyle(iframeContent, null).getPropertyValue(\"line-height\"); // Parse the line height lineHeight \u003d parseFloat(lineHeight); bookDescriptionAvailableHeight \u003d Math.round(lineHeight * numLines); } else { var bdOffsetTop \u003d document.getElementById(\"bookDescription_feature_div\").offsetTop; var imageBlockOffsetTop \u003d document.getElementById(\"booksImageBlock_feature_div\").offsetTop; var imageBlockHeight \u003d document.getElementById(\"booksImageBlock_feature_div\").offsetHeight; bookDescriptionAvailableHeight \u003d imageBlockOffsetTop + imageBlockHeight -bdOffsetTop - 30; if(bookDescriptionAvailableHeight \u003c minBookDescriptionInitialHeight) { bookDescriptionAvailableHeight \u003d minBookDescriptionInitialHeight; } } \t\t var psTotalHeight \u003d document.getElementById(\"postBodyPS\").offsetHeight; if(psTotalHeight \u003e bookDescriptionAvailableHeight + 30){ if(document.getElementById(\"bdSeeLessPrompt\").style.display \u003d\u003d \"none\"){ document.getElementById(\"outer_postBodyPS\").style.height \u003d bookDescriptionAvailableHeight + \u0027px\u0027; document.getElementById(\"psPlaceHolder\").style.display \u003d\"block\"; document.getElementById(\"bdSeeAllPrompt\").style.display \u003d\"block\"; } else{ document.getElementById(\"outer_postBodyPS\").style.height \u003d psTotalHeight + \u0027px\u0027; } } else{ document.getElementById(\"outer_postBodyPS\").style.height \u003d psTotalHeight + \u0027px\u0027; document.getElementById(\"psPlaceHolder\").style.display \u003d\"none\"; document.getElementById(\"bdSeeAllPrompt\").style.display \u003d\"block\"; document.getElementById(\"bdSeeLessPrompt\").style.display \u003d\"none\"; document.getElementById(\"bdExpanderIcon\").className \u003d document.getElementById(\"bdExpanderIcon\").className.replace(\"rotate\",\"\"); } })(); } options.iframeId \u003d iframeId; options.iframeWrapperId \u003d \"bookDesc_iframe_wrapper\";\toptions.overriddenCSSId \u003d \"bookDesc_override_CSS\";\toptions.encodedIframeContent \u003d bookDescEncodedData;\toptions.initialResizeCallback \u003d resizeCallback;\t\tBookDescriptionIframe \u003d new DynamicIframe(options);\tP.guardFatal(\"bookDescription\", function() {\t BookDescriptionIframe.createIframe();\t}) ();\t \tif ((typeof BookDescriptionIframe !\u003d \u0027undefined\u0027) \u0026\u0026 (BookDescriptionIframe instanceof DynamicIframe)) {\t P.when(\u0027jQuery\u0027).execute(function($) {\t $(window).resize(function() {\t P.guardFatal(\"bookDescription\", function() {\t BookDescriptionIframe.resizeIframe(resizeCallback);\t }) ();\t });\t $(window).bind(\u0027imageResize\u0027, function() {\t P.guardFatal(\"bookDescription\", function() {\t BookDescriptionIframe.resizeIframe(resizeCallback);\t }) ();\t });\t });\t}});\u003c/script\u003e \u003c/div\u003e \u003cdiv id\u003d\"edpIngress_feature_div\" class\u003d\"feature\" data-feature-name\u003d\"edpIngress\"\u003e \u003c/div\u003e \u003cdiv id\u003d\"heroQuickPromoBooksAtf_feature_div\" class\u003d\"feature\" data-feature-name\u003d\"heroQuickPromoBooksAtf\"\u003e \u003cdiv id\u003d\"hero-quick-promo\" class\u003d\"a-row a-spacing-medium\"\u003e \u003chr class\u003d\"a-spacing-medium a-divider-normal\" /\u003e \u003c!--wlhqp--\u003e \u003cdiv class\u003d\"qpImage\"\u003e \u003ca href\u003d\"/gp/redirect.html/ref\u003damb_link_475358582_1/154-5469300-3928621?ie\u003dUTF8\u0026amp;location\u003dhttp%3A%2F%2Fwww.omnivoracious.com%2F\u0026amp;source\u003dstandards\u0026amp;token\u003dDEF1D0758E667C928314D8D6645CBCC4A8DBCD89\u0026amp;pf_rd_m\u003dATVPDKIKX0DER\u0026amp;pf_rd_s\u003dhero-quick-promo-books-atf\u0026amp;pf_rd_r\u003dW7WQP2BDJ090798P990F\u0026amp;pf_rd_t\u003d201\u0026amp;pf_rd_p\u003d2546723202\u0026amp;pf_rd_i\u003d1628391340\"\u003e\u003cimg src\u003d\"https://images-na.ssl-images-amazon.com/images/G/01/img15/books/other/17002_books_ABR-hqp_75x75._SL75_CB330842061_.jpg\" width\u003d\"75\" align\u003d\"left\" alt\u003d\"The Amazon Book Review\" height\u003d\"75\" border\u003d\"0\" /\u003e\u003c/a\u003e \u003c/div\u003e \u003cspan class\u003d\"qpHeadline\"\u003eThe Amazon Book Review\u003c/span\u003e \u003cbr /\u003e Author interviews, book reviews, editors picks, and more. \u003ca href\u003d\"/gp/redirect.html/ref\u003damb_link_475358582_3/154-5469300-3928621?ie\u003dUTF8\u0026amp;location\u003dhttp%3A%2F%2Fwww.omnivoracious.com%2F\u0026amp;source\u003dstandards\u0026amp;token\u003dDEF1D0758E667C928314D8D6645CBCC4A8DBCD89\u0026amp;pf_rd_m\u003dATVPDKIKX0DER\u0026amp;pf_rd_s\u003dhero-quick-promo-books-atf\u0026amp;pf_rd_r\u003dW7WQP2BDJ090798P990F\u0026amp;pf_rd_t\u003d201\u0026amp;pf_rd_p\u003d2546723202\u0026amp;pf_rd_i\u003d1628391340\"\u003eRead it now\u003c/a\u003e \u003c/div\u003e \u003cdiv style\u003d\"clear:left; margin-bottom:5px\"\u003e\u003c/div\u003e \u003c/div\u003e \u003cdiv id\u003d\"promotionalBundle_feature_div\" class\u003d\"feature\" data-feature-name\u003d\"promotionalBundle\"\u003e \u003c/div\u003e \u003cdiv id\u003d\"andonCord_feature_div\" class\u003d\"feature\" data-feature-name\u003d\"andonCord\"\u003e \u003c/div\u003e \u003cdiv id\u003d\"OosRecommendation_feature_div\" class\u003d\"feature\" data-feature-name\u003d\"OosRecommendation\"\u003e \u003c/div\u003e \u003c/div\u003e"],"defaultValue":"","isMetaAttribute":false,"extractedValuesCount":1},{"name":"IMAGE","dataType":"STRING","values":[""],"defaultValue":"","isMetaAttribute":false,"extractedValuesCount":0},{"name":"SELLER_INFO","dataType":"STRING","values":[""],"defaultValue":"","isMetaAttribute":false,"extractedValuesCount":0},{"name":"PRODUCT_DETAILS","dataType":"STRING","values":["\u003cdiv id\u003d\"detail-bullets\"\u003e \u003ca name\u003d\"productDetails\" id\u003d\"productDetails\"\u003e\u003c/a\u003e \u003chr noshade\u003d\"noshade\" size\u003d\"1\" class\u003d\"bucketDivider\" /\u003e \u003ctable cellpadding\u003d\"0\" cellspacing\u003d\"0\" border\u003d\"0\" id\u003d\"productDetailsTable\"\u003e \u003ctbody\u003e \u003ctr\u003e \u003ctd class\u003d\"bucket\"\u003e \u003ch2\u003eProduct Details\u003c/h2\u003e \u003cdiv class\u003d\"content\"\u003e \u003cul\u003e \u003cli\u003e\u003cb\u003ePaperback:\u003c/b\u003e 200 pages\u003c/li\u003e \u003cli\u003e\u003cb\u003ePublisher:\u003c/b\u003e Xulon Press (July 22, 2013)\u003c/li\u003e \u003cli\u003e\u003cb\u003eLanguage:\u003c/b\u003e English\u003c/li\u003e \u003cli\u003e\u003cb\u003eISBN-10:\u003c/b\u003e 1628391340\u003c/li\u003e \u003cli\u003e\u003cb\u003eISBN-13:\u003c/b\u003e 978-1628391343\u003c/li\u003e \u003cli\u003e\u003cb\u003e Product Dimensions: \u003c/b\u003e 6.1 x 0.4 x 9.2 inches \u003c/li\u003e \u003cli\u003e\u003cb\u003eShipping Weight:\u003c/b\u003e 10.2 ounces (\u003ca href\u003d\"/gp/help/seller/shipping.html/ref\u003ddp_pd_shipping/154-5469300-3928621?ie\u003dUTF8\u0026amp;asin\u003d1628391340\u0026amp;seller\u003dATVPDKIKX0DER\"\u003eView shipping rates and policies\u003c/a\u003e)\u003c/li\u003e \u003cli\u003e\u003cb\u003eAverage Customer Review:\u003c/b\u003e \u003cscript type\u003d\"text/javascript\"\u003efunction acrPopoverHover(e, h) { if(h) window.acrAsinHover \u003d e; else if(window.acrAsinHover \u003d\u003d e) window.acrAsinHover \u003d null;}function popoverReady(jQuery) { if((typeof jQuery \u003d\u003d\u003d \"undefined\") || (typeof jQuery.fn \u003d\u003d\u003d \"undefined\")) { return; } else if(jQuery.fn.acrPopover) { return; } var popoverConfig \u003d { showOnHover:true, showCloseButton: true, width: null, location:\u0027bottom\u0027, locationAlign:\u0027left\u0027, locationOffset:[-20,0], paddingLeft: 15, paddingBottom: 5, paddingRight: 15, group: \u0027reviewsPopover\u0027, clone:false, hoverHideDelay:300 }; jQuery.fn.acrPopover \u003d function() { return this.each(function() { var $this \u003d jQuery(this); if(!$this.data(\u0027init\u0027) \u0026\u0026 typeof $this.amazonPopoverTrigger \u003d\u003d\u003d \u0027function\u0027) { $this.data(\u0027init\u0027, 1); var getargs \u003d $this.attr(\u0027getargs\u0027); var ajaxURL \u003d \u0027/gp/customer-reviews/common/du/displayHistoPopAjax.html?\u0027 + \u0027\u0026ASIN\u003d\u0027 + $this.attr(\u0027name\u0027) + \u0027\u0026link\u003d1\u0027 + \u0027\u0026seeall\u003d1\u0027 + \u0027\u0026ref\u003d\u0027 + $this.attr(\u0027ref\u0027) + ((typeof getargs !\u003d \u0027undefined\u0027) ? (\u0027\u0026getargs\u003d\u0027 + getargs) : \u0027\u0027); var myConfig \u003d jQuery.extend(true, { destination: ajaxURL }, popoverConfig); $this.amazonPopoverTrigger(myConfig); var w \u003d window.acrAsinHover; if(w \u0026\u0026 jQuery(w).parents(\u0027.asinReviewsSummary\u0027).get(0) \u003d\u003d this) { $this.trigger(\u0027mouseover.amzPopover\u0027); window.acrAsinHover \u003d null; } } }); }; window.reviewHistPopoverConfig \u003d popoverConfig; var jqInit \u003d window.jQueryInitHistoPopovers \u003d function(asin) { if (typeof jQuery(\u0027.acr-popover[name\u003d\u0027 + asin + \u0027]\u0027).acrPopover \u003d\u003d\u003d \u0027function\u0027) { jQuery(\u0027.acr-popover[name\u003d\u0027 + asin + \u0027]\u0027).acrPopover(); } }; window.doInit_average_customer_reviews \u003d jqInit; window.onAjaxUpdate_average_customer_reviews \u003d jqInit; window.onCacheUpdate_average_customer_reviews \u003d jqInit; window.onCacheUpdateReselect_average_customer_reviews \u003d jqInit; if (typeof P !\u003d\u003d \u0027undefined\u0027) { P.when(\u0027amznJQ.criticalFeature\u0027).execute(\"amznJQ.criticalFeatureComplete\", function() { setTimeout(function() { P.register(\u0027acrPopover\u0027, function () { return window.arcPopover; }); }, 10); }); } else if (typeof amznJQ !\u003d\u003d \u0027undefined\u0027) { amznJQ.onCompletion(\u0027amznJQ.criticalFeature\u0027, function() { setTimeout(function() { amznJQ.declareAvailable(\u0027acrPopover\u0027); }, 10); }); }};function acrPopoverReady(jQuery) { jQuery(\u0027.acr-popover,#searchTemplate .asinReviewsSummary\u0027).each(function() { if (typeof jQuery(this).acrPopover \u003d\u003d\u003d \u0027function\u0027) { jQuery(this).acrPopover(); } });};if (typeof P !\u003d\u003d \u0027undefined\u0027) { var jq \u003d null; if (typeof $ !\u003d\u003d \u0027undefined\u0027) { jq \u003d $; } else if (typeof jQuery !\u003d\u003d \u0027undefined\u0027) { jq \u003d jQuery; } if (jq) { P.when(\u0027popover\u0027, \u0027ready\u0027).execute(popoverReady(jq)); P.when(\u0027acrPopover\u0027, \u0027ready\u0027).execute(acrPopoverReady(jq)); }} else if ((typeof amznJQ !\u003d\u003d \u0027undefined\u0027) \u0026\u0026 (typeof jQuery !\u003d\u003d \u0027undefined\u0027)) { amznJQ.onReady(\u0027popover\u0027, popoverReady, jQuery); amznJQ.onReady(\u0027acrPopover\u0027, acrPopoverReady, jQuery);};\u003c/script\u003e \u003cspan class\u003d\"crAvgStars\" style\u003d\"white-space:no-wrap;\"\u003e\u003cspan class\u003d\"asinReviewsSummary acr-popover\" name\u003d\"1628391340\" ref\u003d\"dp_db_cm_cr_acr_pop_\"\u003e \u003ca style\u003d\"cursor:pointer;text-decoration:none\" href\u003d\"https://www.amazon.com/Before-First-Day-Stephen-Hale/product-reviews/1628391340/ref\u003ddp_db_cm_cr_acr_img/154-5469300-3928621?ie\u003dUTF8\u0026amp;showViewpoints\u003d1\" name\u003d\"reviewHistoPop_1628391340__star__\" onmouseover\u003d\"return acrPopoverHover(this,1);\" onmouseout\u003d\"return acrPopoverHover(this,0);\"\u003e\u003cspan class\u003d\"swSprite s_star_5_0 \" title\u003d\"5.0 out of 5 stars\"\u003e\u003cspan\u003e5.0 out of 5 stars\u003c/span\u003e\u003c/span\u003e\u0026nbsp;\u003c/a\u003e\u0026nbsp;\u003cspan class\u003d\"histogramButton\" style\u003d\"margin-left:-3px\"\u003e\u003ca style\u003d\"cursor:pointer\" href\u003d\"https://www.amazon.com/Before-First-Day-Stephen-Hale/product-reviews/1628391340/ref\u003ddp_db_cm_cr_acr_img/154-5469300-3928621?ie\u003dUTF8\u0026amp;showViewpoints\u003d1\" name\u003d\"reviewHistoPop_1628391340__button__\" onmouseover\u003d\"return acrPopoverHover(this,1);\" onmouseout\u003d\"return acrPopoverHover(this,0);\"\u003e\u003cspan class\u003d\"swSprite s_chevron \"\u003e\u003cspan\u003eSee all reviews\u003c/span\u003e\u003c/span\u003e\u0026nbsp;\u003c/a\u003e\u003c/span\u003e\u003c/span\u003e(\u003ca href\u003d\"https://www.amazon.com/Before-First-Day-Stephen-Hale/product-reviews/1628391340/ref\u003ddp_db_cm_cr_acr_txt/154-5469300-3928621?ie\u003dUTF8\u0026amp;showViewpoints\u003d1\"\u003e1 customer review\u003c/a\u003e)\u003c/span\u003e\u003c/li\u003e \u003cli id\u003d\"SalesRank\"\u003e \u003cb\u003eAmazon Best Sellers Rank:\u003c/b\u003e #9,398,863 in Books (\u003ca href\u003d\"https://www.amazon.com/best-sellers-books-Amazon/zgbs/books/ref\u003dpd_dp_ts_b_1/154-5469300-3928621\"\u003eSee Top 100 in Books\u003c/a\u003e) \u003cstyle type\u003d\"text/css\"\u003e.zg_hrsr { margin: 0; padding: 0; list-style-type: none; }.zg_hrsr_item { margin: 0 0 0 10px; }.zg_hrsr_rank { display: inline-block; width: 80px; text-align: right; }\u003c/style\u003e \u003cul class\u003d\"zg_hrsr\"\u003e \u003cli class\u003d\"zg_hrsr_item\"\u003e \u003cspan class\u003d\"zg_hrsr_rank\"\u003e#6112\u003c/span\u003e \u003cspan class\u003d\"zg_hrsr_ladder\"\u003ein\u0026nbsp;\u003ca href\u003d\"https://www.amazon.com/best-sellers-books-Amazon/zgbs/books/ref\u003dpd_zg_hrsr_b_1_1/154-5469300-3928621\"\u003eBooks\u003c/a\u003e \u0026gt; \u003ca href\u003d\"https://www.amazon.com/gp/bestsellers/books/12290/ref\u003dpd_zg_hrsr_b_1_2/154-5469300-3928621\"\u003eChristian Books \u0026amp; Bibles\u003c/a\u003e \u0026gt; \u003ca href\u003d\"https://www.amazon.com/gp/bestsellers/books/172806/ref\u003dpd_zg_hrsr_b_1_3/154-5469300-3928621\"\u003eLiterature \u0026amp; Fiction\u003c/a\u003e \u0026gt; \u003cb\u003e\u003ca href\u003d\"https://www.amazon.com/gp/bestsellers/books/7259433011/ref\u003dpd_zg_hrsr_b_1_4_last/154-5469300-3928621\"\u003eFantasy\u003c/a\u003e\u003c/b\u003e\u003c/span\u003e \u003c/li\u003e \u003cli class\u003d\"zg_hrsr_item\"\u003e \u003cspan class\u003d\"zg_hrsr_rank\"\u003e#105141\u003c/span\u003e \u003cspan class\u003d\"zg_hrsr_ladder\"\u003ein\u0026nbsp;\u003ca href\u003d\"https://www.amazon.com/best-sellers-books-Amazon/zgbs/books/ref\u003dpd_zg_hrsr_b_2_1/154-5469300-3928621\"\u003eBooks\u003c/a\u003e \u0026gt; \u003ca href\u003d\"https://www.amazon.com/gp/bestsellers/books/17/ref\u003dpd_zg_hrsr_b_2_2/154-5469300-3928621\"\u003eLiterature \u0026amp; Fiction\u003c/a\u003e \u0026gt; \u003ca href\u003d\"https://www.amazon.com/gp/bestsellers/books/10134/ref\u003dpd_zg_hrsr_b_2_3/154-5469300-3928621\"\u003eGenre Fiction\u003c/a\u003e \u0026gt; \u003cb\u003e\u003ca href\u003d\"https://www.amazon.com/gp/bestsellers/books/12489/ref\u003dpd_zg_hrsr_b_2_4_last/154-5469300-3928621\"\u003eReligious \u0026amp; Inspirational\u003c/a\u003e\u003c/b\u003e\u003c/span\u003e \u003c/li\u003e \u003cli class\u003d\"zg_hrsr_item\"\u003e \u003cspan class\u003d\"zg_hrsr_rank\"\u003e#1162374\u003c/span\u003e \u003cspan class\u003d\"zg_hrsr_ladder\"\u003ein\u0026nbsp;\u003ca href\u003d\"https://www.amazon.com/best-sellers-books-Amazon/zgbs/books/ref\u003dpd_zg_hrsr_b_3_1/154-5469300-3928621\"\u003eBooks\u003c/a\u003e \u0026gt; \u003cb\u003e\u003ca href\u003d\"https://www.amazon.com/gp/bestsellers/books/22/ref\u003dpd_zg_hrsr_b_3_2_last/154-5469300-3928621\"\u003eReligion \u0026amp; Spirituality\u003c/a\u003e\u003c/b\u003e\u003c/span\u003e \u003c/li\u003e \u003c/ul\u003e \u003c/li\u003e \u003c/ul\u003e \u003cspan class\u003d\"tiny\"\u003e \u003cul class\u003d\"noteBullets\"\u003e \u003c/ul\u003e \u003c/span\u003e \u003c/div\u003e \u003c/td\u003e \u003c/tr\u003e \u003c/tbody\u003e \u003c/table\u003e \u003c/div\u003e"],"defaultValue":"","isMetaAttribute":false,"extractedValuesCount":1},{"name":"TECH_DETAILS","dataType":"STRING","values":[""],"defaultValue":"","isMetaAttribute":false,"extractedValuesCount":0},{"name":"CAPTCHA","dataType":"STRING","values":[""],"defaultValue":"","isMetaAttribute":false,"extractedValuesCount":0},{"name":"PRODUCT_IDENTIFIER","dataType":"STRING","values":[""],"defaultValue":"","isMetaAttribute":false,"extractedValuesCount":0},{"name":"BUYBOX","dataType":"STRING","values":["\u003cdiv id\u003d\"unifiedBuyBox_feature_div\" class\u003d\"feature\" data-feature-name\u003d\"unifiedBuyBox\"\u003e \u003cdiv id\u003d\"combinedBuyBox\" class\u003d\"a-section a-spacing-medium\"\u003e \u003cform method\u003d\"post\" id\u003d\"addToCart\" action\u003d\"/gp/product/handle-buy-box/ref\u003ddp_start-bbf_1_glance\" class\u003d\"a-content\"\u003e \u003cinput type\u003d\"hidden\" id\u003d\"session-id\" name\u003d\"session-id\" value\u003d\"154-5469300-3928621\" /\u003e \u003cinput type\u003d\"hidden\" id\u003d\"ASIN\" name\u003d\"ASIN\" value\u003d\"1628391340\" /\u003e \u003cinput type\u003d\"hidden\" id\u003d\"offerListingID\" name\u003d\"offerListingID\" value\u003d\"P0YRnR9szBbdW1hK5VhfhfwyIoQrnhuEd%2BPy%2BBgCbCzNct9GAa8y5i0LbWFzbHdQ3KzbWuQMLfVSXokLOyYlLfXo60wCnhwcYD%2BQpFk1lcQBrvHvvzu1mw%3D%3D\" /\u003e \u003cinput type\u003d\"hidden\" id\u003d\"isMerchantExclusive\" name\u003d\"isMerchantExclusive\" value\u003d\"0\" /\u003e \u003cinput type\u003d\"hidden\" id\u003d\"merchantID\" name\u003d\"merchantID\" value\u003d\"ATVPDKIKX0DER\" /\u003e \u003cinput type\u003d\"hidden\" id\u003d\"isAddon\" name\u003d\"isAddon\" value\u003d\"0\" /\u003e \u003cinput type\u003d\"hidden\" id\u003d\"nodeID\" name\u003d\"nodeID\" value\u003d\"283155\" /\u003e \u003cinput type\u003d\"hidden\" id\u003d\"sellingCustomerID\" name\u003d\"sellingCustomerID\" value\u003d\"A2R2RITDJNW1Q6\" /\u003e \u003cinput type\u003d\"hidden\" id\u003d\"qid\" name\u003d\"qid\" value\u003d\"\" /\u003e \u003cinput type\u003d\"hidden\" id\u003d\"sr\" name\u003d\"sr\" value\u003d\"\" /\u003e \u003cinput type\u003d\"hidden\" id\u003d\"storeID\" name\u003d\"storeID\" value\u003d\"books\" /\u003e \u003cinput type\u003d\"hidden\" id\u003d\"tagActionCode\" name\u003d\"tagActionCode\" value\u003d\"283155\" /\u003e \u003cinput type\u003d\"hidden\" id\u003d\"viewID\" name\u003d\"viewID\" value\u003d\"glance\" /\u003e \u003cinput type\u003d\"hidden\" id\u003d\"rsid\" name\u003d\"rsid\" value\u003d\"154-5469300-3928621\" /\u003e \u003cinput type\u003d\"hidden\" id\u003d\"sourceCustomerOrgListID\" name\u003d\"sourceCustomerOrgListID\" value\u003d\"\" /\u003e \u003cinput type\u003d\"hidden\" id\u003d\"sourceCustomerOrgListItemID\" name\u003d\"sourceCustomerOrgListItemID\" value\u003d\"\" /\u003e \u003cinput type\u003d\"hidden\" name\u003d\"wlPopCommand\" value\u003d\"\" /\u003e \u003cdiv id\u003d\"buybox\" class\u003d\"a-row a-spacing-medium\"\u003e \u003cdiv id\u003d\"rbbContainer\" class\u003d\"a-box-group dp-accordion\"\u003e \u003cdiv class\u003d\"a-box rbbSection selected dp-accordion-active\"\u003e \u003cdiv class\u003d\"a-box-inner\"\u003e \u003cdiv class\u003d\"a-section a-spacing-none a-padding-none\"\u003e \u003cdiv id\u003d\"buyNewSection\" class\u003d\"rbbHeader dp-accordion-row\"\u003e \u003ci class\u003d\"a-icon a-icon-radio-active\"\u003e\u003c/i\u003e \u003ca class\u003d\"a-link-normal rbbHeaderLink\" href\u003d\"/gp/product/1628391340?selectObb\u003dnew\"\u003e \u003ch5\u003e \u003cdiv class\u003d\"a-row\"\u003e \u003cdiv class\u003d\"a-column a-span4 a-text-left a-nowrap\"\u003e \u003cspan class\u003d\"a-text-bold\"\u003eBuy New\u003c/span\u003e \u003c/div\u003e \u003cdiv class\u003d\"a-column a-span8 a-text-right a-span-last\"\u003e \u003cdiv class\u003d\"inlineBlock-display\"\u003e \u003cspan class\u003d\"a-letter-space\"\u003e\u003c/span\u003e \u003cspan class\u003d\"a-size-medium a-color-price offer-price a-text-normal\"\u003e$15.49\u003c/span\u003e \u003c/div\u003e \u003c/div\u003e \u003c/div\u003e \u003c/h5\u003e \u003c/a\u003e \u003c/div\u003e \u003cdiv id\u003d\"buyNewInner\" class\u003d\"rbbContent dp-accordion-inner\"\u003e \u003cdiv id\u003d\"buyBoxInner\" class\u003d\"a-section a-spacing-none\"\u003e \u003cdiv class\u003d\"a-row\"\u003e \u003cdiv class\u003d\"a-column a-span5\"\u003e \u003cdiv id\u003d\"selectQuantity\" class\u003d\"a-section a-spacing-none a-padding-none\"\u003e \u003cspan class\u003d\"a-declarative\" data-action\u003d\"quantity-dropdown\" data-quantity-dropdown\u003d\"{}\"\u003e \u003cspan class\u003d\"a-dropdown-container\"\u003e\u003clabel for\u003d\"quantity\" class\u003d\"a-native-dropdown\"\u003eQty:\u003c/label\u003e\u003cselect name\u003d\"quantity\" autocomplete\u003d\"off\" id\u003d\"quantity\" tabindex\u003d\"-1\" class\u003d\"a-native-dropdown\"\u003e \u003coption value\u003d\"1\" selected\u003d\"\"\u003e1 \u003c/option\u003e \u003coption value\u003d\"2\"\u003e2 \u003c/option\u003e \u003coption value\u003d\"3\"\u003e3 \u003c/option\u003e \u003coption value\u003d\"4\"\u003e4 \u003c/option\u003e \u003coption value\u003d\"5\"\u003e5 \u003c/option\u003e \u003coption value\u003d\"6\"\u003e6 \u003c/option\u003e \u003coption value\u003d\"7\"\u003e7 \u003c/option\u003e \u003coption value\u003d\"8\"\u003e8 \u003c/option\u003e \u003coption value\u003d\"9\"\u003e9 \u003c/option\u003e \u003coption value\u003d\"10\"\u003e10 \u003c/option\u003e \u003coption value\u003d\"11\"\u003e11 \u003c/option\u003e \u003coption value\u003d\"12\"\u003e12 \u003c/option\u003e \u003coption value\u003d\"13\"\u003e13 \u003c/option\u003e \u003coption value\u003d\"14\"\u003e14 \u003c/option\u003e \u003coption value\u003d\"15\"\u003e15 \u003c/option\u003e \u003coption value\u003d\"16\"\u003e16 \u003c/option\u003e \u003coption value\u003d\"17\"\u003e17 \u003c/option\u003e \u003coption value\u003d\"18\"\u003e18 \u003c/option\u003e \u003coption value\u003d\"19\"\u003e19 \u003c/option\u003e \u003coption value\u003d\"20\"\u003e20 \u003c/option\u003e \u003coption value\u003d\"21\"\u003e21 \u003c/option\u003e \u003coption value\u003d\"22\"\u003e22 \u003c/option\u003e \u003coption value\u003d\"23\"\u003e23 \u003c/option\u003e \u003coption value\u003d\"24\"\u003e24 \u003c/option\u003e \u003coption value\u003d\"25\"\u003e25 \u003c/option\u003e \u003coption value\u003d\"26\"\u003e26 \u003c/option\u003e \u003coption value\u003d\"27\"\u003e27 \u003c/option\u003e \u003coption value\u003d\"28\"\u003e28 \u003c/option\u003e \u003coption value\u003d\"29\"\u003e29 \u003c/option\u003e \u003coption value\u003d\"30\"\u003e30 \u003c/option\u003e \u003c/select\u003e\u003cspan tabindex\u003d\"-1\" class\u003d\"a-button a-button-dropdown a-button-small\"\u003e\u003cspan class\u003d\"a-button-inner\"\u003e\u003cspan class\u003d\"a-button-text a-declarative\" data-action\u003d\"a-dropdown-button\" role\u003d\"button\" tabindex\u003d\"0\" aria-hidden\u003d\"true\"\u003e\u003cspan class\u003d\"a-dropdown-label\"\u003eQty:\u003c/span\u003e\u003cspan class\u003d\"a-dropdown-prompt\"\u003e1\u003c/span\u003e\u003c/span\u003e\u003ci class\u003d\"a-icon a-icon-dropdown\"\u003e\u003c/i\u003e\u003c/span\u003e\u003c/span\u003e\u003c/span\u003e \u003c/span\u003e \u003c/div\u003e \u003c/div\u003e \u003cdiv class\u003d\"a-column a-span7 a-text-right a-span-last\"\u003e \u003cul class\u003d\"a-nostyle a-vertical\"\u003e \u003c/ul\u003e \u003c/div\u003e \u003c/div\u003e \u003c/div\u003e \u003cdiv class\u003d\"a-section a-spacing-small a-spacing-top-micro\"\u003e \u003cdiv class\u003d\"a-row\"\u003e \u003cspan class\u003d\"a-color-base buyboxShippingLabel\"\u003e \u003c!-- MsgId:cfs_free_shipping_eligible_no_popover_link:web --\u003e \u003ca href\u003d\"/gp/help/customer/display.html/ref\u003dmk_sss_dp_1?ie\u003dUTF8\u0026amp;pop-up\u003d1\u0026amp;nodeId\u003d527692\" target\u003d\"AmazonHelp\" onclick\u003d\"return amz_js_PopWin(this.href,\u0027AmazonHelp\u0027,\u0027width\u003d550,height\u003d550,resizable\u003d1,scrollbars\u003d1,toolbar\u003d0,status\u003d0\u0027);\"\u003eFREE Shipping\u003c/a\u003e on orders with at least $25 of books.\u003c/span\u003e \u003c/div\u003e \u003c/div\u003e \u003cdiv class\u003d\"a-section a-spacing-none\"\u003e \u003c/div\u003e \u003cdiv class\u003d\"a-section a-spacing-small\"\u003e \u003cdiv id\u003d\"availability\" class\u003d\"a-section a-spacing-none\"\u003e \u003cspan class\u003d\"a-size-medium a-color-success\"\u003e May take an extra 1-2 days to ship. \u003c/span\u003e \u003c/div\u003e \u003cdiv class\u003d\"a-section a-spacing-none\"\u003e \u003c/div\u003e \u003cdiv id\u003d\"merchant-info\" class\u003d\"a-section a-spacing-mini\"\u003e Ships from and sold by Amazon.com. \u003cspan class\u003d\"\"\u003e Gift-wrap available. \u003c/span\u003e \u003c/div\u003e \u003c/div\u003e \u003cdiv id\u003d\"bbopAndCartBox\" class\u003d\"a-box\"\u003e \u003cdiv class\u003d\"a-box-inner\"\u003e \u003cstyle type\u003d\"text/css\"\u003e#bbop-sbbop-container { margin-bottom: 0px;}\u003c/style\u003e \u003cdiv id\u003d\"bbop-sbbop-container\" class\u003d\"a-section\"\u003e \u003cdiv id\u003d\"sbbop-popover-header\" class\u003d\"a-section a-hidden\"\u003e \u003cdiv class\u003d\"a-box a-alert-inline a-alert-inline-success\"\u003e \u003cdiv class\u003d\"a-box-inner a-alert-container\"\u003e \u003ci class\u003d\"a-icon a-icon-alert\"\u003e\u003c/i\u003e \u003cdiv class\u003d\"a-color-base\"\u003e Before the First Day has been added to your Cart \u003c/div\u003e \u003c/div\u003e \u003c/div\u003e \u003c/div\u003e \u003c/div\u003e \u003cscript type\u003d\"text/javascript\"\u003eif( window.P ){ P.when(\"A\").execute(function(A) { if (window.sbbopLoaded \u003d\u003d\u003d undefined) { window.sbbopLoaded \u003d false; } function getQueryParameterByName(name) { name \u003d name.replace(/[\\[]/, \"\\\\[\").replace(/[\\]]/, \"\\\\]\"); var regex \u003d new RegExp(\"[\\\\?\u0026]\" + name + \"\u003d([^\u0026#]*)\"), results \u003d regex.exec(location.search); return results \u003d\u003d null ? \"\" : decodeURIComponent(results[1].replace(/\\+/g, \" \")); } var hidden, visibilityChange; if (typeof document.hidden !\u003d\u003d \"undefined\") { hidden \u003d \"hidden\"; visibilityChange \u003d \"visibilitychange\"; } else if (typeof document.mozHidden !\u003d\u003d \"undefined\") { hidden \u003d \"mozHidden\"; visibilityChange \u003d \"mozvisibilitychange\"; } else if (typeof document.msHidden !\u003d\u003d \"undefined\") { hidden \u003d \"msHidden\"; visibilityChange \u003d \"msvisibilitychange\"; } else if (typeof document.webkitHidden !\u003d\u003d \"undefined\") { hidden \u003d \"webkitHidden\"; visibilityChange \u003d \"webkitvisibilitychange\"; } var loadFeatures \u003d function() { var $ \u003d A.$, $container \u003d $(\"#bbop-sbbop-container\"); var data \u003d { \"ASIN\" : \"1628391340\", \"merchantID\" : \"ATVPDKIKX0DER\", \"bbopruleID\" : \"Acquisition_AddToCart_PrimeBasicFreeTrialUpsellEligible\", \"sbbopruleID\" : \"Acquisition_AddToCart_PrimeBasicFreeTrialUpsellEligible\", \"deliveryOptions\" : \"[next,std-n-us,sss-us,second]\", \"preorder\" : \"false\", \"releaseDateDeliveryEligible\" : \"false\" }; var scope \u003d \"bbopAjaxCall\"; if(typeof uet \u003d\u003d\u003d \u0027function\u0027) { uet(\u0027bb\u0027, scope, {wb: 1}); } $.get(\"/gp/product/du/bbop-ms3-ajax-endpoint.html\", data, function(result) { $container.append(result); if(typeof uet \u003d\u003d\u003d \u0027function\u0027 \u0026\u0026 typeof uex \u003d\u003d\u003d \u0027function\u0027) { uet( \"cf\", scope, {wb: 1} ); uex( \"ld\", scope, {wb: 1} ); } if (getQueryParameterByName(\"checkBBOP\") \u003d\u003d\u003d \"true\") { P.when(\"a-modal\", \"ready\").execute(function(modal) { $(\u0027#bbop-check-box\u0027).click(); }); } loadSBBOP(); }); var loadSBBOP \u003d function(){ P.when(\"a-modal\", \"ready\").execute(function(modal) { if(!window.sbbopLoaded){ var sbbop_modal \u003d createModal(modal); if (sbbop_modal !\u003d\u003d undefined) { $(document.body).delegate(\u0027#add-to-cart-button\u0027,\u0027click.sbbop\u0027, function(event){ $(document.body).undelegate(\u0027#add-to-cart-button\u0027,\u0027click.sbbop\u0027); var $sbbop \u003d $(\"#sbbop-container\"), $sbbopContent \u003d $sbbop.find(\"#sbbop-popover-content\"), $sbbopHeader \u003d $container.find(\"#sbbop-popover-header\"), $sbbopFooter \u003d $sbbop.find(\"#sbbop-popover-footer\"), isAsinizationCase \u003d $sbbop.find(\".is-sbbop-asinization-usecase\").length \u003e 0, $bbopCheckbox \u003d $(\u0027#bbop-check-box\u0027), $atcButton \u003d $(\u0027#add-to-cart-button\u0027), atcName \u003d $atcButton.attr(\"name\"), hasSingleYesButton \u003d ($(\u0027#sbbop-yes-button\u0027).length \u003d\u003d\u003d 1), hasSingleNoButton \u003d ($(\u0027#sbbop-no-button\u0027).length \u003d\u003d\u003d 1), validAtcButton \u003d ($atcButton.length \u003d\u003d\u003d 1 \u0026\u0026 atcName !\u003d \"submit.add-to-cart-prime-buy-box.x\"), validSbbopContent \u003d ($sbbopContent.length \u003d\u003d\u003d 1 \u0026\u0026 $sbbopHeader.length \u003d\u003d\u003d 1 \u0026\u0026 $sbbopFooter.length \u003d\u003d\u003d 1), validSbbopButtons \u003d (hasSingleYesButton \u0026\u0026 hasSingleNoButton), bbopIsNotChecked \u003d !$bbopCheckbox.is(\":checked\"); if (validAtcButton \u0026\u0026 validSbbopContent \u0026\u0026 validSbbopButtons \u0026\u0026 bbopIsNotChecked) { $atcButton.bind(\u0027click.sbbop\u0027, function(){return false;}); $(document.body).delegate(\"#sbbop-yes-button\", \"click\", function(event){ if(isAsinizationCase){ $bbopCheckbox.prop(\"checked\",true); logMetric(\"CSM_prime_surprise_BBOP_yes_button_HO\"); } else { $atcButton.attr(\"name\", \"submit.add-to-cart-prime-buy-box.x\"); logMetric(\"CSM_prime_surprise_BBOP_yes_button\"); } safeATCClick(); }); $(document.body).delegate(\"#sbbop-no-button\", \"click\", function(event){ if(isAsinizationCase) { logMetric(\"CSM_prime_surprise_BBOP_no_button_HO\"); } else { logMetric(\"CSM_prime_surprise_BBOP_no_button\"); } safeATCClick(); }); $(document.body).delegate(\".a-popover\", \"keydown\", function(event){ if(event.keyCode \u003d\u003d \"27\"){ event.preventDefault(); } }); A.on(\"a:popover:ajaxFail:sbbop_modal\", safeATCClick); A.on(\"a:popover:hide:sbbop_modal\", function(){ if(isAsinizationCase) { logMetric(\"CSM_prime_surprise_BBOP_close_button_HO\"); } else { logMetric(\"CSM_prime_surprise_BBOP_close_button\"); } safeATCClick(); }); A.on(\"a:popover:beforeShow:sbbop_modal\", function(){ setEventEmbu(); }); sbbop_modal.show(); if(isAsinizationCase) { logMetric(\"CSM_prime_surprise_BBOP_presentation_HO\"); } else { logMetric(\"CSM_prime_surprise_BBOP_presentation\"); } return false; } }); } window.sbbopLoaded \u003d true; } }); }; var createModal \u003d function(modal) { var $sbbop \u003d $(\"#sbbop-container\"), $sbbopContent \u003d $sbbop.find(\"#sbbop-popover-content\"), $sbbopHeader \u003d $container.find(\"#sbbop-popover-header\"), $sbbopFooter \u003d $sbbop.find(\"#sbbop-popover-footer\"); if($sbbopContent.length \u003d\u003d\u003d 1 \u0026\u0026 $sbbopHeader.length \u003d\u003d\u003d 1 \u0026\u0026 $sbbopFooter.length \u003d\u003d\u003d 1){ sbbopWidth \u003d parseInt(($sbbop.find(\"#sbbop-popover\")).attr(\"popover-width\"), 10); return modal.create($sbbop, { \"inlineContent\": $sbbopContent.html(), \"header\": $sbbopHeader.html(), \"footer\": $sbbopFooter.html(), \"width\": sbbopWidth, \"activate\": \"onclick\", \"name\": \"sbbop_modal\" }); } return undefined; } var logMetric \u003d function(customTag){ if(window.ue \u0026\u0026 ue.tag) { ue.tag(customTag, \"surpriseBBOP\") } }; var postBackMetrics \u003d function(){ if (window.ue){ uex(\"ld\", \"surpriseBBOP\"); } }; var setEventEmbu \u003d function(){ var eventData \u003d { \"eventCode\" : \"106\" }; $.get(\"/gp/prime/utility/record-embu-event.html\", eventData); }; var safeATCClick \u003d function() { postBackMetrics(); $(\u0027#add-to-cart-button\u0027).unbind(\u0027click.sbbop\u0027) .click() .bind(\u0027click.sbbop\u0027,function(){return false}); }; } var handleVisibilityChange \u003d function() { if (!document[hidden]) { document.removeEventListener(visibilityChange, handleVisibilityChange); loadFeatures(); } } if (typeof document.addEventListener \u003d\u003d\u003d \"undefined\" || typeof document[hidden] \u003d\u003d\u003d \"undefined\" || !document[hidden]) { loadFeatures(); } else { document.addEventListener(visibilityChange, handleVisibilityChange); } });}\u003c/script\u003e \u003cdiv class\u003d\"a-button-stack\"\u003e \u003cspan id\u003d\"submit.add-to-cart\" class\u003d\"a-button a-spacing-small a-button-primary a-button-icon\"\u003e\u003cspan class\u003d\"a-button-inner\"\u003e\u003ci class\u003d\"a-icon a-icon-cart\"\u003e\u003c/i\u003e\u003cinput id\u003d\"add-to-cart-button\" name\u003d\"submit.add-to-cart\" title\u003d\"Add to Shopping Cart\" data-hover\u003d\"Select \u0026lt;b\u0026gt;__dims__\u0026lt;/b\u0026gt; from the left\u0026lt;br\u0026gt; to add to Shopping Cart\" class\u003d\"a-button-input\" type\u003d\"submit\" value\u003d\"Add to Cart\" aria-labelledby\u003d\"submit.add-to-cart-announce\" /\u003e\u003cspan id\u003d\"submit.add-to-cart-announce\" class\u003d\"a-button-text\" aria-hidden\u003d\"true\"\u003eAdd to Cart\u003c/span\u003e\u003c/span\u003e\u003c/span\u003e \u003c/div\u003e \u003c/div\u003e \u003c/div\u003e \u003cdiv class\u003d\"a-row a-spacing-none\"\u003e \u003cdiv id\u003d\"oneClickSignIn\" class\u003d\"a-section a-spacing-none\"\u003e \u003cdiv class\u003d\"a-divider a-divider-break a-spacing-micro\"\u003e \u003ch5\u003e\u003ca href\u003d\"/gp/product/utility/edit-one-click-pref.html?ie\u003dUTF8\u0026amp;query\u003dselectObb%3dnew\u0026amp;returnPath\u003d%2fgp%2fproduct%2f1628391340\"\u003e\u003cspan class\u003d\"a-size-mini\"\u003eTurn on 1-Click ordering for this browser\u003c/span\u003e\u003c/a\u003e\u003c/h5\u003e \u003c/div\u003e \u003c/div\u003e \u003c/div\u003e \u003cdiv class\u003d\"a-row\"\u003e \u003cdiv id\u003d\"dpFastTrack_feature_div\" data-feature-name\u003d\"dpFastTrack\" data-template-name\u003d\"dpFastTrack\" class\u003d\"a-section a-spacing-none a-spacing-top-small feature\"\u003e \u003c!-- We only want padding, if the weblab is on--\u003e \u003cdiv id\u003d\"fast-track\" class\u003d\"a-section a-spacing-none\"\u003e \u003cinput type\u003d\"hidden\" id\u003d\"ftSelectAsin\" value\u003d\"1628391340\" /\u003e \u003cinput type\u003d\"hidden\" id\u003d\"ftSelectMerchant\" value\u003d\"ATVPDKIKX0DER\" /\u003e \u003cdiv id\u003d\"fast-track-message\" class\u003d\"a-section a-spacing-none\"\u003e \u003c/div\u003e \u003cscript type\u003d\"text/javascript\"\u003e P.when(\"A\", \"jQuery\").execute(function(A, $) { $(\"#quantity\").live(\"change\", function (event) { if (event.updateFTOnQuantityChange) { return; } event.updateFTOnQuantityChange \u003d 1; var quantity \u003d $(this).val(); var asin \u003d $(\"#ftSelectAsin\").val(); var merchantId \u003d $(\"#ftSelectMerchant\").val(); if (!asin || !merchantId) { return; } var params \u003d []; params.push(\"asin\u003d\" + asin); params.push(\"quantity\u003d\" + quantity); params.push(\"merchantId\u003d\" + merchantId); $.ajax({ type: \"POST\", url: \"/gp/product/features/dp-fast-track/udp-ajax-handler/get-quantity-update-message.html?ie\u003dUTF8\", contentType: \u0027application/x-www-form-urlencoded;charset\u003dutf-8\u0027, data: params.join(\u0027\u0026\u0027), dataType: \"html\", success: function(objResponse) { if (objResponse !\u003d null \u0026\u0026 objResponse !\u003d \"\") { $(\"#fast-track-message\").replaceWith(objResponse); } } }); return; }); });\u003c/script\u003e \u003c/div\u003e \u003c/div\u003e \u003c/div\u003e \u003cdiv id\u003d\"unifiedLocation_feature_div\" data-feature-name\u003d\"unifiedLocation\" data-template-name\u003d\"unifiedLocation\" class\u003d\"a-section feature unifiedLocationMarginBottomClass\"\u003e \u003cscript type\u003d\"text/javascript\"\u003e P.now(\"LOCUX_DETALPAGE_51079_T1\").execute(function(LOCUX_DETALPAGE_51079_T1) { if (!LOCUX_DETALPAGE_51079_T1) { P.declare(\"LOCUX_DETALPAGE_51079_T1\", {}); } }); \u003c/script\u003e \u003cdiv class\u003d\"a-section a-spacing-none\"\u003e \u003chr class\u003d\"a-divider-normal\" /\u003e \u003cdiv class\u003d\"a-section a-spacing-mini\"\u003e \u003cspan class\u003d\"a-text-bold\"\u003e Ship to: \u003c/span\u003e \u003c/div\u003e \u003cdiv class\u003d\"a-section a-spacing-none\"\u003e \u003cspan class\u003d\"a-declarative\" data-action\u003d\"a-popover\" data-a-popover\u003d\"{\u0026quot;closeButton\u0026quot;:\u0026quot;false\u0026quot;,\u0026quot;name\u0026quot;:\u0026quot;LUXAddressSelector\u0026quot;,\u0026quot;activate\u0026quot;:\u0026quot;onclick\u0026quot;}\"\u003e \u003ca href\u003d\"javascript:void(0)\" class\u003d\"a-popover-trigger a-declarative\"\u003e \u003cspan class\u003d\"a-color-base lux-location-label\"\u003e DALLAS, TX 75201 \u003c/span\u003e \u003ci class\u003d\"a-icon a-icon-popover\"\u003e\u003c/i\u003e\u003c/a\u003e \u003c/span\u003e \u003c/div\u003e \u003cdiv class\u003d\"a-popover-preload\" id\u003d\"a-popover-LUXAddressSelector\"\u003e \u003c!-- Section to show to unrecognized customers (instead of address selector) --\u003e \u003cdiv id\u003d\"unifiedLocationTitleNoAddress\" class\u003d\"a-section a-spacing-small a-spacing-top-small a-text-center\"\u003e \u003cspan class\u003d\"a-text-bold\"\u003e To see addresses, please \u003c/span\u003e \u003cbr /\u003e \u003cspan id\u003d\"unifiedLocationSignIn\" class\u003d\"a-button a-spacing-top-base a-button-primary\"\u003e\u003cspan class\u003d\"a-button-inner\"\u003e\u003cinput class\u003d\"a-button-input\" type\u003d\"submit\" aria-labelledby\u003d\"unifiedLocationSignIn-announce\" /\u003e\u003cspan id\u003d\"unifiedLocationSignIn-announce\" class\u003d\"a-button-text\" aria-hidden\u003d\"true\"\u003e Sign in \u003c/span\u003e\u003c/span\u003e\u003c/span\u003e \u003c/div\u003e \u003c!-- Place holder for anchoring the address selector --\u003e \u003cdiv id\u003d\"lux-address-selector-anchor\" class\u003d\"a-section a-spacing-none\"\u003e\u003c/div\u003e \u003cdiv class\u003d\"a-section a-spacing-none lux-address-selector-divider\"\u003e \u003c/div\u003e \u003cdiv id\u003d\"unifiedLocationAddrInputDiv\" class\u003d\"a-section a-spacing-none\"\u003e \u003cdiv class\u003d\"a-divider a-divider-break lux-zip-selector-divider\"\u003e \u003ch5\u003eor\u003c/h5\u003e \u003c/div\u003e \u003clabel class\u003d\"a-text-bold\"\u003e Use this location: \u003c/label\u003e \u003cdiv class\u003d\"a-row a-spacing-top-micro\" role\u003d\"form\"\u003e \u003cdiv class\u003d\"a-column a-span8\" role\u003d\"form\"\u003e \u003cinput type\u003d\"text\" maxlength\u003d\"5\" id\u003d\"unifiedLocationAddrInput\" placeholder\u003d\"Enter US zip\" class\u003d\"a-input-text a-span12\" /\u003e \u003c/div\u003e \u003cdiv class\u003d\"a-column a-span4 a-span-last\" role\u003d\"button\"\u003e \u003cspan id\u003d\"unifiedLocationAddrUpdate\" class\u003d\"a-button a-button-span12\"\u003e\u003cspan class\u003d\"a-button-inner\"\u003e\u003cinput class\u003d\"a-button-input\" type\u003d\"submit\" aria-labelledby\u003d\"unifiedLocationAddrUpdate-announce\" /\u003e\u003cspan id\u003d\"unifiedLocationAddrUpdate-announce\" class\u003d\"a-button-text\" aria-hidden\u003d\"true\"\u003e Update \u003c/span\u003e\u003c/span\u003e\u003c/span\u003e \u003c/div\u003e \u003c/div\u003e \u003cdiv id\u003d\"unifiedLocationInvalidInputAlert\" class\u003d\"a-row a-spacing-top-small aok-hidden\"\u003e \u003cspan class\u003d\"a-color-error\"\u003e Please enter a valid US zip code. \u003c/span\u003e \u003c/div\u003e \u003cdiv id\u003d\"LUXInvalidZipCodeAlert\" class\u003d\"a-row a-spacing-top-small aok-hidden\" role\u003d\"alert\"\u003e \u003cdiv class\u003d\"a-box a-alert-inline a-alert-inline-error a-spacing-none\"\u003e \u003cdiv class\u003d\"a-box-inner a-alert-container\"\u003e \u003ci class\u003d\"a-icon a-icon-alert\"\u003e\u003c/i\u003e \u003cdiv class\u003d\"a-alert-content\"\u003e Please enter a valid US zip code. \u003c/div\u003e \u003c/div\u003e \u003c/div\u003e \u003c/div\u003e \u003cdiv id\u003d\"LUXMilitaryZipCodeAlert\" class\u003d\"a-row a-spacing-top-small aok-hidden\" role\u003d\"alert\"\u003e \u003cdiv class\u003d\"a-box a-alert-inline a-alert-inline-info a-spacing-none\"\u003e \u003cdiv class\u003d\"a-box-inner a-alert-container\"\u003e \u003ci class\u003d\"a-icon a-icon-alert\"\u003e\u003c/i\u003e \u003cdiv class\u003d\"a-alert-content\"\u003e Shipping to a APO/FPO/DPO? Please add the address to your address book. Make sure you include the unit and box numbers (if assigned). \u003c/div\u003e \u003c/div\u003e \u003c/div\u003e \u003c/div\u003e \u003c/div\u003e \u003cdiv class\u003d\"a-divider a-divider-break a-spacing-base a-spacing-top-base lux-country-selector-divider\"\u003e \u003ch5\u003eor\u003c/h5\u003e \u003c/div\u003e \u003cdiv id\u003d\"unifiedLocationCountrySelectionDiv\" class\u003d\"a-row a-spacing-none a-spacing-top-small\"\u003e \u003c/div\u003e \u003c/div\u003e \u003cscript type\u003d\"text/javascript\"\u003e var shouldPopulateCountrySelector \u003d true; P.now(\"LOCUX_DETAILPAGE_68144_T1\").execute(function(LOCUX_DETAILPAGE_68144_T1) { if(LOCUX_DETAILPAGE_68144_T1) { shouldPopulateCountrySelector \u003d true; } }); var fullPageRefresh \u003d false; P.now(\"LOCUX_FULLPAGE_REFRESH\").execute(function(LOCUX_FULLPAGE_REFRESH) { if(LOCUX_FULLPAGE_REFRESH) { fullPageRefresh \u003d true; } }); P.now(\"LUXContext\").execute(function(LUXContext) { if (!LUXContext) { P.declare(\"LUXContext\", { isRecognizedCustomer: false, deviceType: \"web\", isOneClick: false, clientPage: \"DPX\", shouldPopulateCountrySelector: shouldPopulateCountrySelector, fullPageRefresh: fullPageRefresh }); } }); P.when(\"LUXController\").execute(function(LUXController) { LUXController.init({ addressLabel : \"\", addressId : \"\", obfuscatedId : \"\", countryCode : \"US\", zipCode : \"75201\", state : \"TX\", city : \"DALLAS\", district : \"\", locationType : \"IP2LOCATION\" }); }); \u003c/script\u003e \u003c/div\u003e \u003cinput type\u003d\"hidden\" name\u003d\"dropdown-selection\" value\u003d\"add-new\" id\u003d\"unifiedLocationAddress\" data-addnewaddress\u003d\"add-new\" /\u003e \u003cscript type\u003d\"text/javascript\"\u003e P.now(\"LUXDPOnly\").execute(function(DP) { DP || P.declare(\"LUXDPOnly\", {}); }); P.when(\"LUXDPRefreshController\").execute(function(Controller) { Controller.init({ asin : \"1628391340\", merchantId : \"ATVPDKIKX0DER\", deviceType : \"web\", smid : \"\", exMerchId : \"\", wdg: \"book_display_on_website\", oneClick : false, addressId : \"\", obfuscatedId : \"\", city : \"DALLAS\", state : \"TX\", countryCode : \"US\", zipCode : \"75201\", district : \"\", useTwisterRefresh : \"C\", updateAddressForNon1Click : \"T1\", relatedRequestId : \"W7WQP2BDJ090798P990F\" }); }); \u003c/script\u003e \u003c/div\u003e \u003cdiv class\u003d\"a-row\"\u003e \u003cdiv id\u003d\"holidayAvailabilityMessage_feature_div\" data-feature-name\u003d\"holidayAvailabilityMessage\" data-template-name\u003d\"holidayAvailabilityMessage\" class\u003d\"a-section a-spacing-top-small feature\"\u003e \u003c/div\u003e \u003c/div\u003e \u003c/div\u003e \u003c/div\u003e \u003c/div\u003e \u003c/div\u003e \u003cdiv class\u003d\"a-box rbbSection unselected\"\u003e \u003cdiv class\u003d\"a-box-inner\"\u003e \u003cdiv class\u003d\"a-section a-spacing-none a-padding-none\"\u003e \u003cdiv id\u003d\"usedBuySection\" class\u003d\"rbbHeader dp-accordion-row\"\u003e \u003ci class\u003d\"a-icon a-icon-radio-inactive\"\u003e\u003c/i\u003e \u003ca class\u003d\"a-link-normal rbbHeaderLink\" href\u003d\"/gp/product/1628391340?selectObb\u003dused\"\u003e \u003ch5\u003e \u003cdiv class\u003d\"a-row\"\u003e \u003cdiv class\u003d\"a-column a-span4 a-text-left a-nowrap\"\u003e \u003cspan class\u003d\"a-text-bold\"\u003eBuy Used\u003c/span\u003e \u003c/div\u003e \u003cdiv class\u003d\"a-column a-span8 a-text-right a-span-last\"\u003e \u003cdiv class\u003d\"inlineBlock-display\"\u003e \u003cspan class\u003d\"a-letter-space\"\u003e\u003c/span\u003e \u003cspan class\u003d\"a-color-base offer-price a-text-normal\"\u003e$12.94\u003c/span\u003e \u003c/div\u003e \u003c/div\u003e \u003c/div\u003e \u003c/h5\u003e \u003c/a\u003e \u003c/div\u003e \u003cdiv id\u003d\"usedbuyBox\" class\u003d\"a-section rbbContent dp-accordion-inner\"\u003e \u003cinput type\u003d\"hidden\" id\u003d\"usedMerchantID\" name\u003d\"usedMerchantID\" value\u003d\"A3QP1Q1OVJQM8C\" /\u003e \u003cinput type\u003d\"hidden\" id\u003d\"usedOfferListingID\" name\u003d\"usedOfferListingID\" value\u003d\"NTBvnTZwEVBK9r5EhFUVCKqW6Ex8W7A9MKFOezgEz53HNLJ1CYQZ5WjObDRVVYtNWyo7jZdkdj3gRmS7XVPmO9aFJSyenmNJC5DvMY93A4w3j5T5kE%2FqxoqIOlVilcqdkAiQYyDDFjK7HhbKPpL86g%3D%3D\" /\u003e \u003cinput type\u003d\"hidden\" id\u003d\"usedSellingCustomerID\" name\u003d\"usedSellingCustomerID\" value\u003d\"A3QP1Q1OVJQM8C\" /\u003e \u003cdiv class\u003d\"a-section a-spacing-mini\"\u003e \u003cdiv class\u003d\"a-row\"\u003e +\u0026nbsp;$3.99\u0026nbsp;shipping \u003c/div\u003e \u003c/div\u003e \u003cdiv class\u003d\"a-section a-spacing-base\"\u003e \u003cdiv class\u003d\"a-row\"\u003e \u003cstrong\u003e Used: Like New \u003c/strong\u003e \u003cspan class\u003d\"a-size-base\"\u003e \u003cspan class\u003d\"a-color-tertiary\"\u003e | \u003c/span\u003e \u003ca id\u003d\"usedItemConditionInfoLink\" class\u003d\"a-link-normal\" href\u003d\"#\"\u003e Details \u003c/a\u003e \u003c/span\u003e \u003c/div\u003e \u003cdiv class\u003d\"a-row\"\u003e Sold by \u003ca class\u003d\"a-link-normal\" href\u003d\"/gp/help/seller/at-a-glance.html?ie\u003dUTF8\u0026amp;seller\u003dA3QP1Q1OVJQM8C\"\u003ePrepbooks\u003c/a\u003e \u003c/div\u003e \u003c/div\u003e \u003cdiv class\u003d\"a-popover-preload\" id\u003d\"a-popover-usedItemConditionDetailsPopover\"\u003e \u003cdiv class\u003d\"a-section a-spacing-micro\"\u003e \u003cspan class\u003d\"a-size-mini\"\u003e \u003cstrong\u003eCondition:\u003c/strong\u003e Used: Like New \u003c/span\u003e \u003c/div\u003e \u003cdiv class\u003d\"a-section a-spacing-micro\"\u003e \u003cspan class\u003d\"a-size-mini\"\u003e \u003cstrong\u003eComment:\u003c/strong\u003e Multiple Quantities. Fast International Shipping. Tracking Number Provided on all Orders. Quick Order Processing. Ships from Multiple US Locations based on Location and Availability. Satisfaction Guaranteed. \u003c/span\u003e \u003c/div\u003e \u003c/div\u003e \u003cdiv class\u003d\"a-section accessCode-spacing\"\u003e \u003c/div\u003e \u003cdiv class\u003d\"a-button-stack\"\u003e \u003cspan id\u003d\"submit.add-to-cart-ubb\" class\u003d\"a-button a-spacing-small a-button-primary a-button-icon\"\u003e\u003cspan class\u003d\"a-button-inner\"\u003e\u003ci class\u003d\"a-icon a-icon-cart\"\u003e\u003c/i\u003e\u003cinput id\u003d\"add-to-cart-button-ubb\" name\u003d\"submit.add-to-cart-ubb\" title\u003d\"Add to Shopping Cart\" data-hover\u003d\"Select \u0026lt;b\u0026gt;__dims__\u0026lt;/b\u0026gt; from the left\u0026lt;br\u0026gt; to add to Shopping Cart\" class\u003d\"a-button-input\" type\u003d\"submit\" value\u003d\"Add to Cart\" aria-labelledby\u003d\"submit.add-to-cart-ubb-announce\" /\u003e\u003cspan id\u003d\"submit.add-to-cart-ubb-announce\" class\u003d\"a-button-text\" aria-hidden\u003d\"true\"\u003eAdd to Cart\u003c/span\u003e\u003c/span\u003e\u003c/span\u003e \u003c/div\u003e \u003cdiv class\u003d\"a-section a-spacing-none a-text-center\"\u003e \u003cdiv class\u003d\"a-row a-spacing-none\"\u003e \u003cdiv id\u003d\"oneClickSignInUBB\" class\u003d\"a-section a-spacing-none\"\u003e \u003cdiv class\u003d\"a-divider a-divider-break a-spacing-micro\"\u003e \u003ch5\u003e\u003ca href\u003d\"/gp/product/utility/edit-one-click-pref.html?ie\u003dUTF8\u0026amp;query\u003dselectObb%3dused\u0026amp;returnPath\u003d%2fgp%2fproduct%2f1628391340\"\u003e\u003cspan class\u003d\"a-size-mini\"\u003eTurn on 1-Click ordering for this browser\u003c/span\u003e\u003c/a\u003e\u003c/h5\u003e \u003c/div\u003e \u003c/div\u003e \u003c/div\u003e \u003c/div\u003e \u003c/div\u003e \u003c/div\u003e \u003c/div\u003e \u003c/div\u003e \u003c/div\u003e \u003c/div\u003e \u003cdiv id\u003d\"wishlistButtonStack\" class\u003d\"a-button-stack\"\u003e \u003cdiv id\u003d\"add-to-wishlist-button-group\" data-hover\u003d\"\u0026lt;!-- If PartialItemStateWeblab is true then, showing different Add-to-wish-list tool-tip message which is consistent with Add-to-Cart tool tip message. --\u0026gt; To Add to Your List, choose from options to the left\" class\u003d\"a-button-group a-declarative a-spacing-none\" data-action\u003d\"a-button-group\" role\u003d\"radiogroup\"\u003e \u003cspan id\u003d\"wishListMainButton\" class\u003d\"a-button a-button-group-first a-spacing-none\" role\u003d\"radio\" aria-labelledby\u003d\"wishListMainButton-announce\"\u003e\u003cspan class\u003d\"a-button-inner\"\u003e\u003cinput id\u003d\"add-to-wishlist-button-submit\" name\u003d\"submit.add-to-registry.wishlist\" title\u003d\"Add to List\" data-action\u003d\"atwl-splitbutton-main\" data-hover\u003d\"\u0026lt;!-- If PartialItemStateWeblab is true then, showing different Add-to-wish-list tool-tip message which is consistent with Add-to-Cart tool tip message. --\u0026gt; To Add to Your List, choose from options to the left\" class\u003d\"a-button-input a-declarative\" type\u003d\"submit\" aria-labelledby\u003d\"wishListMainButton-announce\" /\u003e\u003cspan id\u003d\"wishListMainButton-announce\" class\u003d\"a-button-text a-text-left\" aria-hidden\u003d\"true\"\u003e Add to List \u003c/span\u003e\u003c/span\u003e\u003c/span\u003e \u003c/div\u003e \u003cdiv id\u003d\"atwl-dd-spinner-holder\" class\u003d\"a-section a-hidden\"\u003e \u003cdiv class\u003d\"a-row a-dropdown\"\u003e \u003cdiv class\u003d\"a-section a-popover-wrapper\"\u003e \u003cdiv class\u003d\"a-section a-text-center a-popover-inner\"\u003e \u003cdiv class\u003d\"a-box a-popover-loading\"\u003e \u003cdiv class\u003d\"a-box-inner\"\u003e \u003c/div\u003e \u003c/div\u003e \u003c/div\u003e \u003c/div\u003e \u003c/div\u003e \u003c/div\u003e \u003cdiv id\u003d\"atwl-dd-error-holder\" class\u003d\"a-section a-hidden\"\u003e \u003cdiv class\u003d\"a-section a-dropdown\"\u003e \u003cdiv class\u003d\"a-section a-popover-wrapper\"\u003e \u003cdiv class\u003d\"a-section a-spacing-base a-padding-base a-text-left a-popover-inner\"\u003e \u003ch3 class\u003d\"a-color-error\"\u003e Sorry, there was a problem. \u003c/h3\u003e \u003cspan\u003e There was an error retrieving your Wish Lists. Please try again. \u003c/span\u003e \u003c/div\u003e \u003c/div\u003e \u003c/div\u003e \u003c/div\u003e \u003cdiv id\u003d\"atwl-dd-unavail-holder\" class\u003d\"a-section a-hidden\"\u003e \u003cdiv class\u003d\"a-section a-dropdown\"\u003e \u003cdiv class\u003d\"a-section a-popover-wrapper\"\u003e \u003cdiv class\u003d\"a-section a-spacing-base a-padding-base a-text-left a-popover-inner\"\u003e \u003ch3 class\u003d\"a-color-error\"\u003e Sorry, there was a problem. \u003c/h3\u003e \u003cspan\u003e List unavailable. \u003c/span\u003e \u003c/div\u003e \u003c/div\u003e \u003c/div\u003e \u003c/div\u003e \u003cscript type\u003d\"a-state\" data-a-state\u003d\"{\u0026quot;key\u0026quot;:\u0026quot;atwl\u0026quot;}\"\u003e{\"hzPopover\":true,\"wishlistButtonId\":\"add-to-wishlist-button\",\"dropDownHtml\":\"\",\"wishlistButtonSubmitId\":\"add-to-wishlist-button-submit\",\"maxAjaxFailureCount\":\"3\",\"asin\":\"1628391340\"}\u003c/script\u003e \u003cscript type\u003d\"a-state\" data-a-state\u003d\"{\u0026quot;key\u0026quot;:\u0026quot;jumpstart-success-modal-state\u0026quot;}\"\u003e{\"showDPHeadJavaScript\":false,\"showBabyRegSuccessModal\":false,\"addSuccessModalHeader\":\"Added to your Baby Registry\",\"hasBabyReg\":false}\u003c/script\u003e \u003cinput type\u003d\"hidden\" name\u003d\"\" value\u003d\"true\" id\u003d\"is-fully-selected-baby-reg\" /\u003e \u003c/div\u003e \u003cscript type\u003d\"a-state\" data-a-state\u003d\"{\u0026quot;key\u0026quot;:\u0026quot;popoverState\u0026quot;}\"\u003e{\"formId\":\"addToCart\",\"showWishListDropDown\":false,\"babyRegistryId\":\"add-to-registry-baby-button\",\"showBabyRegDropdown\":false,\"wishlistPopoverWidth\":232,\"isAddToWishListDropDownAuiEnabled\":true,\"showPopover\":false,\"isBabyRegistryBrowseScriptDeprecated\":true}\u003c/script\u003e \u003c/form\u003e \u003c/div\u003e \u003c/div\u003e"],"defaultValue":"","isMetaAttribute":false,"extractedValuesCount":1},{"name":"BREADCRUMB","dataType":"STRING","values":["\u003cdiv id\u003d\"wayfinding-breadcrumbs_container\" class\u003d\"a-section a-spacing-none a-padding-medium\"\u003e \u003cdiv id\u003d\"wayfinding-breadcrumbs_feature_div\" class\u003d\"a-subheader a-breadcrumb feature\" data-feature-name\u003d\"wayfinding-breadcrumbs\"\u003e \u003cul class\u003d\"a-unordered-list a-horizontal a-size-small\"\u003e \u003cli\u003e\u003cspan class\u003d\"a-list-item\"\u003e \u003ca class\u003d\"a-link-normal a-color-tertiary\" href\u003d\"/books-used-books-textbooks/b/ref\u003ddp_bc_1/154-5469300-3928621?ie\u003dUTF8\u0026amp;node\u003d283155\"\u003e Books \u003c/a\u003e \u003c/span\u003e\u003c/li\u003e \u003cli class\u003d\"a-breadcrumb-divider\"\u003e\u003cspan class\u003d\"a-list-item a-color-tertiary\"\u003e › \u003c/span\u003e\u003c/li\u003e \u003cli\u003e\u003cspan class\u003d\"a-list-item\"\u003e \u003ca class\u003d\"a-link-normal a-color-tertiary\" href\u003d\"/Christianity-Religion-Spirituality-Books/b/ref\u003ddp_bc_2/154-5469300-3928621?ie\u003dUTF8\u0026amp;node\u003d12290\"\u003e Christian Books \u0026amp; Bibles \u003c/a\u003e \u003c/span\u003e\u003c/li\u003e \u003cli class\u003d\"a-breadcrumb-divider\"\u003e\u003cspan class\u003d\"a-list-item a-color-tertiary\"\u003e › \u003c/span\u003e\u003c/li\u003e \u003cli\u003e\u003cspan class\u003d\"a-list-item\"\u003e \u003ca class\u003d\"a-link-normal a-color-tertiary\" href\u003d\"/Fiction-Poetry-Christianity-Books/b/ref\u003ddp_bc_3/154-5469300-3928621?ie\u003dUTF8\u0026amp;node\u003d172806\"\u003e Literature \u0026amp; Fiction \u003c/a\u003e \u003c/span\u003e\u003c/li\u003e \u003c/ul\u003e \u003c/div\u003e \u003c/div\u003e"],"defaultValue":"","isMetaAttribute":false,"extractedValuesCount":1},{"name":"PARENT_URL","dataType":"STRING","values":[""],"isMetaAttribute":true,"extractedValuesCount":0},{"name":"HTTP_STATUS","dataType":"STRING","values":["200"],"isMetaAttribute":true,"extractedValuesCount":0},{"name":"CRAWLMETADATA","dataType":"STRING","values":["{\"crawlType\":\"PRODUCT\",\"retailerName\":\"amazon_us\",\"site\":\"US\",\"metaCategId\":267,\"leafCategId\":171228,\"productCrawlMetadata\":{\"retailerProductId\":\"1628391340\",\"referencePrice\":0.0,\"referenceShippingPrice\":0.0,\"selectors\":[]},\"tracker\":{\"crawlRequestCreateTS\":1486483774000}}"],"isMetaAttribute":true,"extractedValuesCount":0},{"name":"CRAWL_TIMESTAMP","dataType":"STRING","values":["1486484044649"],"isMetaAttribute":true,"extractedValuesCount":0},{"name":"URL","dataType":"STRING","values":["https://www.amazon.com/gp/product/1628391340/ref\u003dcm_cr_pr_product_top"],"isMetaAttribute":true,"extractedValuesCount":0}]}]} \ No newline at end of file diff --git a/measure/src/test/scala/org/apache/griffin/measure/utils/HdfsUtilTest.scala b/measure/src/test/scala/org/apache/griffin/measure/utils/HdfsUtilTest.scala index 90db32af1..6a672d5bf 100644 --- a/measure/src/test/scala/org/apache/griffin/measure/utils/HdfsUtilTest.scala +++ b/measure/src/test/scala/org/apache/griffin/measure/utils/HdfsUtilTest.scala @@ -44,10 +44,10 @@ class HdfsUtilTest extends FunSuite with Matchers with BeforeAndAfter { conf2.addResource(new Path("file:///Users/lliu13/test/hadoop/hdfs-site.xml")) private val dfs2 = FileSystem.get(conf2) - val conf = new SparkConf().setAppName("test_hdfs").setMaster("local[*]") - val sparkContext = new SparkContext(conf) - sparkContext.setLogLevel("WARN") - val sqlContext = new HiveContext(sparkContext) +// val conf = new SparkConf().setAppName("test_hdfs").setMaster("local[*]") +// val sparkContext = new SparkContext(conf) +// sparkContext.setLogLevel("WARN") +// val sqlContext = new HiveContext(sparkContext) def listSubPaths(dfs: FileSystem, dirPath: String, subType: String, fullPath: Boolean = false): Iterable[String] = { val path = new Path(dirPath) @@ -75,58 +75,58 @@ class HdfsUtilTest extends FunSuite with Matchers with BeforeAndAfter { if (parentPath.endsWith(seprator)) parentPath + fileName else parentPath + seprator + fileName } - test ("test multiple hdfs") { - val list1 = listSubPaths(dfs1, "/", "dir", false) - println(list1) - - val list2 = listSubPaths(dfs2, "/", "dir", false) - println(list2) - - val path1 = "/depth/discovery_file_sample.txt" - val istream1 = dfs1.open(new Path(path1)) - val reader1 = new BufferedReader(new InputStreamReader(istream1)) - val seq1 = scala.collection.mutable.MutableList[String]() - try { - var line = reader1.readLine() - while (line != null) { - val arr = line.split("\u0007") - seq1 ++= arr - line = reader1.readLine() - } - } finally { - reader1.close() - istream1.close() - } - -// val scanner = new java.util.Scanner(istream1,"UTF-8").useDelimiter("\u0007") -// val theString = if (scanner.hasNext()) scanner.next() else "" -// println(theString) -// scanner.close() - - println(seq1.size) - println(seq1.take(10)) - seq1.take(10).foreach(println) - -// val path2 = "/griffin/json/env.json" -// val istream2 = dfs2.open(new Path(path2)) -// val reader2 = new BufferedReader(new InputStreamReader(istream2)) -// val seq2 = scala.collection.mutable.MutableList[String]() +// test ("test multiple hdfs") { +// val list1 = listSubPaths(dfs1, "/", "dir", false) +// println(list1) +// +// val list2 = listSubPaths(dfs2, "/", "dir", false) +// println(list2) +// +// val path1 = "/depth/discovery_file_sample.txt" +// val istream1 = dfs1.open(new Path(path1)) +// val reader1 = new BufferedReader(new InputStreamReader(istream1)) +// val seq1 = scala.collection.mutable.MutableList[String]() // try { -// var line = reader2.readLine() +// var line = reader1.readLine() // while (line != null) { -// line = reader2.readLine() -// seq2 += line -// } -// } catch { -// case e: Throwable => { -// println("error in reading") +// val arr = line.split("\u0007") +// seq1 ++= arr +// line = reader1.readLine() // } // } finally { -// reader2.close() -// istream2.close() +// reader1.close() +// istream1.close() // } -// println(seq2.size) -// println(seq2.take(10)) - } +// +//// val scanner = new java.util.Scanner(istream1,"UTF-8").useDelimiter("\u0007") +//// val theString = if (scanner.hasNext()) scanner.next() else "" +//// println(theString) +//// scanner.close() +// +// println(seq1.size) +// println(seq1.take(10)) +// seq1.take(10).foreach(println) +// +//// val path2 = "/griffin/json/env.json" +//// val istream2 = dfs2.open(new Path(path2)) +//// val reader2 = new BufferedReader(new InputStreamReader(istream2)) +//// val seq2 = scala.collection.mutable.MutableList[String]() +//// try { +//// var line = reader2.readLine() +//// while (line != null) { +//// line = reader2.readLine() +//// seq2 += line +//// } +//// } catch { +//// case e: Throwable => { +//// println("error in reading") +//// } +//// } finally { +//// reader2.close() +//// istream2.close() +//// } +//// println(seq2.size) +//// println(seq2.take(10)) +// } } From dd91bb56b19f47c00b2943264ad22f54aa251b75 Mon Sep 17 00:00:00 2001 From: Lionel Liu Date: Tue, 10 Oct 2017 15:36:28 +0800 Subject: [PATCH 099/111] data source name in from clause bug fix --- griffin-doc/dsl-guide.md | 35 ++++++++++--------- .../measure/rule/dsl/parser/BasicParser.scala | 22 +++++++----- .../test/resources/config-test-profiling.json | 1 + .../rule/dsl/parser/BasicParserTest.scala | 7 ++-- 4 files changed, 39 insertions(+), 26 deletions(-) diff --git a/griffin-doc/dsl-guide.md b/griffin-doc/dsl-guide.md index 40a37a435..a6604a78d 100644 --- a/griffin-doc/dsl-guide.md +++ b/griffin-doc/dsl-guide.md @@ -32,7 +32,7 @@ Griffin DSL is SQL-like, case insensitive, and easy to learn. - `null, nan, true, false` - `not, and, or` - `in, between, like, is` -- `as, where, group, by, having, order, desc, asc, limit` +- `select, from, as, where, group, by, having, order, desc, asc, limit` ### Operators - `!, &&, ||, =, !=, <, >, <=, >=, <>` @@ -42,41 +42,41 @@ Griffin DSL is SQL-like, case insensitive, and easy to learn. ### Literals - **string**: any string surrounded with a pair of " or ', with escape charactor \ if any request. - e.g. `"test", 'string 1', "hello \" world \" "` + e.g. `"test"`, `'string 1'`, `"hello \" world \" "` - **number**: double or integer number. - e.g. `123, 33.5` + e.g. `123`, `33.5` - **time**: a integer with unit in a string, will be translated to a integer number in millisecond. - e.g. `3d, 5h, 4ms` + e.g. `3d`, `5h`, `4ms` - **boolean**: boolean value directly. - e.g. `true, false` + e.g. `true`, `false` ### Selections - **selection head**: data source name. - e.g. `source, target` + e.g. `source`, `target`, `` `my table name` `` - **all field selection**: * or with data source name ahead. - e.g. `*, source.*, target.*` + e.g. `*`, `source.*`, `target.*` - **field selection**: field name or with data source name ahead. - e.g. `source.age, target.name, user_id` + e.g. `source.age`, `target.name`, `user_id` - **index selection**: interget between square brackets "[]" with field name ahead. e.g. `source.attributes[3]` - **function selection**: function name with brackets "()", with field name ahead or not. - e.g. `count(*), *.count(), source.user_id.count(), max(source.age)` + e.g. `count(*)`, `*.count()`, `source.user_id.count()`, `max(source.age)` - **alias**: declare an alias after a selection. - e.g. `source.user_id as id, target.user_name as name` + e.g. `source.user_id as id`, `target.user_name as name` ### Math expressions - **math factor**: literal or function or selection or math exression with brackets. - e.g. `123, max(1, 2, 3, 4), source.age, (source.age + 13)` + e.g. `123`, `max(1, 2, 3, 4)`, `source.age`, `(source.age + 13)` - **unary math expression**: unary math operator with factor. e.g. `-(100 - source.score)` - **binary math expression**: math factors with binary math operators. - e.g. `source.age + 13, score * 2 + ratio` + e.g. `source.age + 13`, `score * 2 + ratio` ### Logical expression - **in**: in clause like sql. e.g. `source.country in ("USA", "CHN", "RSA")` - **between**: between clause like sql. - e.g. `source.age between 3 and 30, source.age between (3, 30)` + e.g. `source.age between 3 and 30`, `source.age between (3, 30)` - **like**: like clause like sql. e.g. `source.name like "%abc%"` - **is null**: is null operator like sql. @@ -86,7 +86,7 @@ Griffin DSL is SQL-like, case insensitive, and easy to learn. - **logical factor**: math expression or logical expressions above or other logical expressions with brackets. e.g. `(source.user_id = target.user_id AND source.age > target.age)` - **unary logical expression**: unary logical operator with factor. - e.g. `NOT source.has_data` + e.g. `NOT source.has_data`, `!(source.age = target.age)` - **binary logical expression**: logical factors with binary logical operators, including `and`, `or` and comparison operators. e.g. `source.age = target.age OR source.ticket = target.tck` @@ -97,7 +97,10 @@ Griffin DSL is SQL-like, case insensitive, and easy to learn. ### Function - **argument**: expression. - **function**: function name with arguments between brackets. - e.g. `max(source.age, target.age), count(*)` + e.g. `max(source.age, target.age)`, `count(*)` ### Clause -- **select clause**: \ No newline at end of file +- **select clause**: the result columns like sql select clause, we can ignore the word "select" in Griffin DSL. + e.g. `select user_id.count(), age.max()`, `source.user_id.count(), source.age.min()` +- **from clause**: the table name like sql from clause, we can ignore this clause by configoring the data source name. + e.g. `from source`, ``from `target` `` \ No newline at end of file diff --git a/measure/src/main/scala/org/apache/griffin/measure/rule/dsl/parser/BasicParser.scala b/measure/src/main/scala/org/apache/griffin/measure/rule/dsl/parser/BasicParser.scala index f55b1f84b..7d9646dc0 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/rule/dsl/parser/BasicParser.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/rule/dsl/parser/BasicParser.scala @@ -86,7 +86,12 @@ trait BasicParser extends JavaTokenParsers with Serializable { * = */ - protected def genNamesParser(names: Seq[String]): Parser[String] = { + protected def genDataSourceNamesParser(names: Seq[String]): Parser[String] = { + names.reverse.map { + fn => s"""(?i)`${fn}`|${fn}""".r: Parser[String] + }.reduce(_ | _) + } + protected def genFunctionNamesParser(names: Seq[String]): Parser[String] = { names.reverse.map { fn => s"""(?i)${fn}""".r: Parser[String] }.reduce(_ | _) @@ -143,9 +148,9 @@ trait BasicParser extends JavaTokenParsers with Serializable { object Strings { def AnyString: Parser[String] = """"(?:[^\"]|\")*"""".r | """'(?:[^']|\')*'""".r def UQuoteTableFieldName: Parser[String] = """`(?:[^`]|[\\][`])*`""".r - def TableFieldName: Parser[String] = UQuoteTableFieldName | """[a-zA-Z_]\w*""".r - def DataSourceName: Parser[String] = genNamesParser(dataSourceNames) - def FunctionName: Parser[String] = genNamesParser(functionNames) + def FieldName: Parser[String] = UQuoteTableFieldName | """[a-zA-Z_]\w*""".r + def DataSourceName: Parser[String] = genDataSourceNamesParser(dataSourceNames) + def FunctionName: Parser[String] = genFunctionNamesParser(functionNames) def IntegerNumber: Parser[String] = """[+\-]?\d+""".r def DoubleNumber: Parser[String] = """[+\-]?(\.\d+|\d+\.\d*)""".r @@ -191,20 +196,20 @@ trait BasicParser extends JavaTokenParsers with Serializable { DataSourceHeadExpr(_) } | function ^^ { OtherHeadExpr(_) - } | TableFieldName ^^ { + } | FieldName ^^ { FieldNameHeadExpr(_) } | ALLSL ^^ { _ => ALLSelectHeadExpr() } def selector: Parser[SelectExpr] = functionSelect | allFieldsSelect | fieldSelect | indexSelect def allFieldsSelect: Parser[AllFieldsSelectExpr] = DOT ~> ALLSL ^^ { _ => AllFieldsSelectExpr() } - def fieldSelect: Parser[FieldSelectExpr] = DOT ~> TableFieldName ^^ { FieldSelectExpr(_) } + def fieldSelect: Parser[FieldSelectExpr] = DOT ~> FieldName ^^ { FieldSelectExpr(_) } def indexSelect: Parser[IndexSelectExpr] = LSQBR ~> argument <~ RSQBR ^^ { IndexSelectExpr(_) } def functionSelect: Parser[FunctionSelectExpr] = DOT ~ FunctionName ~ LBR ~ repsep(argument, COMMA) ~ RBR ^^ { case _ ~ name ~ _ ~ args ~ _ => FunctionSelectExpr(name, args) } - def asAlias: Parser[String] = AS ~> TableFieldName + def asAlias: Parser[String] = AS ~> FieldName /** * -- math expr -- @@ -310,6 +315,7 @@ trait BasicParser extends JavaTokenParsers with Serializable { * -- clauses -- * = [, ]* * = + * = ("data source name registered") * = * = [ ]? * = [ ]? @@ -318,7 +324,7 @@ trait BasicParser extends JavaTokenParsers with Serializable { */ def selectClause: Parser[SelectClause] = opt(SELECT) ~> rep1sep(expression, COMMA) ^^ { SelectClause(_) } - def fromClause: Parser[FromClause] = FROM ~> TableFieldName ^^ { ds => FromClause(trim(ds)) } + def fromClause: Parser[FromClause] = FROM ~> DataSourceName ^^ { ds => FromClause(trim(ds)) } def whereClause: Parser[WhereClause] = WHERE ~> expression ^^ { WhereClause(_) } def havingClause: Parser[Expr] = HAVING ~> expression def groupbyClause: Parser[GroupbyClause] = GROUP ~ BY ~ rep1sep(expression, COMMA) ~ opt(havingClause) ^^ { diff --git a/measure/src/test/resources/config-test-profiling.json b/measure/src/test/resources/config-test-profiling.json index 7c16f24a9..6cfe8a58b 100644 --- a/measure/src/test/resources/config-test-profiling.json +++ b/measure/src/test/resources/config-test-profiling.json @@ -25,6 +25,7 @@ "dq.type": "profiling", "rule": "select user_id as id, user_id.count() as cnt from source group by user_id order by cnt desc, id desc limit 3", "details": { + "source": "source", "profiling": { "name": "count", "persist.type": "metric" diff --git a/measure/src/test/scala/org/apache/griffin/measure/rule/dsl/parser/BasicParserTest.scala b/measure/src/test/scala/org/apache/griffin/measure/rule/dsl/parser/BasicParserTest.scala index 823554d3f..a1b9a83c5 100644 --- a/measure/src/test/scala/org/apache/griffin/measure/rule/dsl/parser/BasicParserTest.scala +++ b/measure/src/test/scala/org/apache/griffin/measure/rule/dsl/parser/BasicParserTest.scala @@ -164,11 +164,13 @@ class BasicParserTest extends FunSuite with Matchers with BeforeAndAfter { test ("test expression") { val rule3 = "source.age + 2 * 5 + target.offset" val result3 = parser.parseAll(parser.expression, rule3) + println(result3) result3.successful should be (true) result3.get.desc should be ("source.age + 2 * 5 + target.offset") val rule4 = "(source.age + 2) * (5 + target.offset)" val result4 = parser.parseAll(parser.expression, rule4) + println(result4) result4.successful should be (true) result4.get.desc should be ("(source.age + 2) * (5 + target.offset)") @@ -203,14 +205,15 @@ class BasicParserTest extends FunSuite with Matchers with BeforeAndAfter { } test ("select clause") { - val rule = "select source.user_id, item, source.func()" + val rule = "select `source`.user_id, item, `source`.age.func()" val result = parser.parseAll(parser.selectClause, rule) + println(result) result.successful should be (true) println(result.get.desc) } test ("from clause") { - val rule = "from source" + val rule = "from `source`" val result = parser.parseAll(parser.fromClause, rule) result.successful should be (true) println(result.get.desc) From 7d7cf52229ab47a2bc3c5ff634575152fc60a4d8 Mon Sep 17 00:00:00 2001 From: Lionel Liu Date: Wed, 11 Oct 2017 10:38:14 +0800 Subject: [PATCH 100/111] git status --- griffin-doc/dsl-guide.md | 81 ++++++++++++++++++- ...onfig-test-accuracy-streaming-multids.json | 2 - .../config-test-accuracy-streaming.json | 2 - 3 files changed, 78 insertions(+), 7 deletions(-) diff --git a/griffin-doc/dsl-guide.md b/griffin-doc/dsl-guide.md index a6604a78d..c0ed332a2 100644 --- a/griffin-doc/dsl-guide.md +++ b/griffin-doc/dsl-guide.md @@ -16,6 +16,7 @@ KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License. --> + # Apache Griffin DSL Guide Griffin DSL is designed for DQ measurement, as a SQL-like language, trying to describe the DQ domain request. @@ -101,6 +102,80 @@ Griffin DSL is SQL-like, case insensitive, and easy to learn. ### Clause - **select clause**: the result columns like sql select clause, we can ignore the word "select" in Griffin DSL. - e.g. `select user_id.count(), age.max()`, `source.user_id.count(), source.age.min()` -- **from clause**: the table name like sql from clause, we can ignore this clause by configoring the data source name. - e.g. `from source`, ``from `target` `` \ No newline at end of file + e.g. `select user_id.count(), age.max() as max`, `source.user_id.count() as cnt, source.age.min()` +- **from clause**: the table name like sql from clause, in which the data source name must be one of data source names or the output table name of the former rule steps, we can ignore this clause by configoring the data source name. + e.g. `from source`, ``from `target` `` +- **where clause**: the filter condition like sql where clause, optional. + e.g. `where source.id = target.id and source.age = target.age` +- **group-by clause**: like the group-by clause in sql, optional. Optional having clause could be following. + e.g. `group by cntry`, `group by gender having count(*) > 50` +- **order-by clause**: like the order-by clause, optional. + e.g. `order by name`, `order by first_name desc, age asc` +- **limit clause**: like the limit clause in sql, optional. + e.g. `limit 5` + +### Accuracy Rule +Accuracy rule expression in Griffin DSL is a logical expression, telling the mapping relation between data sources. + e.g. `source.id = target.id and source.name = target.name and source.age between (target.age, target.age + 5)` + +### Profiling Rule +Profiling rule expression in Griffin DSL is a sql-like expression, with select clause ahead, following optional from clause, where clause, group-by clause, order-by clause, limit clause in order. + e.g. `source.gender, source.id.count() where source.age > 20 group by source.gender`, `select country, max(age), min(age), count(*) as cnt from source group by country order by cnt desc limit 5` + +## Griffin DSL translation to SQL +Griffin DSL is defined for DQ measurement, to describe DQ domain problem. +Actually, in Griffin, we get Griffin DSL rules, translate them into spark-sql rules for calculation in spark-sql engine. +In DQ domain, there're multiple dimensions, we need to translate them in different ways. + +### Accuracy +For accuracy, we need to get the match count between source and target, the rule describes the mapping relation between data sources. Griffin needs to translate the dsl rule into multiple sql rules. +For example, the dsl rule is `source.id = target.id and source.name = target.name`, which represents the match condition of accuracy. After the translation, the sql rules are as below: +- **get miss items from source**: `SELECT source.* FROM source LEFT JOIN target ON coalesce(source.id, '') = coalesce(target.id, '') and coalesce(source.name, '') = coalesce(target.name, '') WHERE (NOT (source.id IS NULL AND source.name IS NULL)) AND (target.id IS NULL AND target.name IS NULL)`, save as table `miss_items`. +- **get miss count**: `SELECT COUNT(*) AS miss FROM miss_items`, save as table `miss_count`. +- **get total count from source**: `SELECT COUNT(*) AS total FROM source`, save as table `total_count`. +- **get accuracy metric**: `SELECT miss_count.miss AS miss, total_count.total AS total, (total_count.total - miss_count.miss) AS matched FROM miss_count FULL JOIN total_count`, save as table `accuracy`. + +After the translation, the metrics will be persisted in table `accuracy`. + +### Profiling +For profiling, the request is always the aggregation function of data, the rule is mainly the same as sql, but only supporting `select`, `from`, `where`, `group-by`, `having`, `order-by`, `limit` clauses, which can describe most of the profiling requests. If any complicate request, you can use sql rule directly to describe it. +For example, the dsl rule is `source.cntry, source.id.count(), source.age.max() group by source.cntry`, which represents the profiling requests. After the translation, the sql rule is as below: +- **profiling sql rule**: `SELECT source.cntry, count(source.id), max(source.age) FROM source GROUP BY source.cntry`, save as table `profiling`. + +After the translation, the metrics will be persisted in table `profiling`. + +## ALternative Rules +You can simply use Griffin DSL rule to describe your problem in DQ domain, for some complicate requirement, you can also use some alternative rules supported by Griffin. + +### Spark sql +Griffin supports spark-sql directly, you can write rule in sql like this: +``` +{ + "dsl.type": "spark-sql", + "name": "source", + "rule": "SELECT count(id) AS cnt, max(timestamp) AS fresh_time FROM source" +} +``` +Griffin will calculate it in spark-sql engine directly. + +### Data frame operation +Griffin supports some other operations on data frame in spark, like converting json string data frame into extracted data frame with extracted object schema. For example: +``` +{ + "dsl.type": "df-opr", + "name": "ext_source", + "rule": "from_json", + "details": { + "df.name": "json_source" + } +} +``` +Griffin will do the operation to extract json strings. +Actually, you can also extend the df-opr engine and df-opr adaptor in Griffin to support more types of data frame operations. + +## Tips +Griffin engine runs on spark, it might works in two phases, pre-proc phase and run phase. +- **Pre-proc phase**: Griffin calculates data source directly, to get appropriate data format, as a preparation for DQ calculation. In this phase, you can use df-opr and spark-sql rules. +After preparation, to support streaming DQ calculation, a timestamp column will be added in each row of data, so the data frame in run phase contains an extra column named "__tmst". +- **Run phase**: Griffin calculates with prepared data, to get the DQ metrics. In this phase, you can use griffin-dsl, spark-sql rules, and a part of df-opr rules. +For griffin-dsl rule, griffin translates it into spark-sql rule with a group-by condition for column "__tmst", it's useful for especially streaming DQ calculation. But for spark-sql rule, griffin use it directly, you need to add the "__tmst" column in your spark-sql rule explicitly, or you can't get correct metrics result after calculation. \ No newline at end of file diff --git a/measure/src/test/resources/config-test-accuracy-streaming-multids.json b/measure/src/test/resources/config-test-accuracy-streaming-multids.json index 18532b0ad..2107ff632 100644 --- a/measure/src/test/resources/config-test-accuracy-streaming-multids.json +++ b/measure/src/test/resources/config-test-accuracy-streaming-multids.json @@ -51,7 +51,6 @@ "dsl.type": "df-opr", "name": "${s1}", "rule": "from_json", - "persist.type": "cache", "details": { "df.name": "${this}" } @@ -93,7 +92,6 @@ "dsl.type": "df-opr", "name": "${t1}", "rule": "from_json", - "persist.type": "cache", "details": { "df.name": "${this}" } diff --git a/measure/src/test/resources/config-test-accuracy-streaming.json b/measure/src/test/resources/config-test-accuracy-streaming.json index 276f8dd70..453f87edc 100644 --- a/measure/src/test/resources/config-test-accuracy-streaming.json +++ b/measure/src/test/resources/config-test-accuracy-streaming.json @@ -26,7 +26,6 @@ "dsl.type": "df-opr", "name": "${s1}", "rule": "from_json", - "persist.type": "cache", "details": { "df.name": "${this}" } @@ -68,7 +67,6 @@ "dsl.type": "df-opr", "name": "${t1}", "rule": "from_json", - "persist.type": "cache", "details": { "df.name": "${this}" } From de7a821f9205dec819bfe25c228f399bf274bffe Mon Sep 17 00:00:00 2001 From: Lionel Liu Date: Wed, 11 Oct 2017 13:16:54 +0800 Subject: [PATCH 101/111] doc --- griffin-doc/dsl-guide.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/griffin-doc/dsl-guide.md b/griffin-doc/dsl-guide.md index c0ed332a2..c668cb9f4 100644 --- a/griffin-doc/dsl-guide.md +++ b/griffin-doc/dsl-guide.md @@ -155,7 +155,7 @@ Griffin supports spark-sql directly, you can write rule in sql like this: "name": "source", "rule": "SELECT count(id) AS cnt, max(timestamp) AS fresh_time FROM source" } -``` +``` Griffin will calculate it in spark-sql engine directly. ### Data frame operation @@ -169,7 +169,7 @@ Griffin supports some other operations on data frame in spark, like converting j "df.name": "json_source" } } -``` +``` Griffin will do the operation to extract json strings. Actually, you can also extend the df-opr engine and df-opr adaptor in Griffin to support more types of data frame operations. From 722bcf796e9310e807126dd483f04770eb5548fb Mon Sep 17 00:00:00 2001 From: Lionel Liu Date: Wed, 11 Oct 2017 13:17:14 +0800 Subject: [PATCH 102/111] doc --- griffin-doc/measure-configuration-guide.md | 77 ++++++++++++++++++++++ 1 file changed, 77 insertions(+) create mode 100644 griffin-doc/measure-configuration-guide.md diff --git a/griffin-doc/measure-configuration-guide.md b/griffin-doc/measure-configuration-guide.md new file mode 100644 index 000000000..69cbcf704 --- /dev/null +++ b/griffin-doc/measure-configuration-guide.md @@ -0,0 +1,77 @@ + + +# Griffin Measure Configuration Guide +Griffin measure module needs two configuration files to define the parameters of execution, one is for environment, the other is for dq job. + +## Environment Parameters +``` +{ + "spark": { + "log.level": "WARN", + "checkpoint.dir": "hdfs:///griffin/streaming/cp", + "batch.interval": "5s", + "process.interval": "30s", + "config": { + "spark.task.maxFailures": 5, + "spark.streaming.kafkaMaxRatePerPartition": 1000, + "spark.streaming.concurrentJobs": 4 + } + }, + + "persist": [ + { + "type": "log", + "config": { + "max.log.lines": 100 + } + }, { + "type": "hdfs", + "config": { + "path": "hdfs:///griffin/streaming/persist", + "max.persist.lines": 10000, + "max.lines.per.file": 10000 + } + } + ], + + "info.cache": [ + { + "type": "zk", + "config": { + "hosts": ":2181", + "namespace": "griffin/infocache", + "lock.path": "lock", + "mode": "persist", + "init.clear": true, + "close.clear": false + } + } + ] +} +``` +Above lists environment parameters. + +- **spark**: This field configures spark and spark streaming parameters. +- **persist**: This field configures the metrics persist parameters, multiple persist ways are supported. +- **info.cache**: This field configures the information cache parameters, multiple cache ways are supported. It is only for streaming dq case. + + + +## DQ Job Parameters From ffb818aa58ef9c330c0bb88731bac8f2975298d8 Mon Sep 17 00:00:00 2001 From: Lionel Liu Date: Wed, 11 Oct 2017 15:38:32 +0800 Subject: [PATCH 103/111] ignore ui tmp --- .gitignore | 1 + griffin-doc/measure-configuration-guide.md | 162 +++++++++++++++++++++ 2 files changed, 163 insertions(+) create mode 100644 griffin-doc/measure-configuration-guide.md diff --git a/.gitignore b/.gitignore index ad52fe537..ee478d874 100644 --- a/.gitignore +++ b/.gitignore @@ -31,6 +31,7 @@ ui/bower_components/* ui/node_modules/* ui/debug.log ui/package-lock.json +ui/tmp derby.log metastore_db diff --git a/griffin-doc/measure-configuration-guide.md b/griffin-doc/measure-configuration-guide.md new file mode 100644 index 000000000..9337831a7 --- /dev/null +++ b/griffin-doc/measure-configuration-guide.md @@ -0,0 +1,162 @@ + + +# Griffin Measure Configuration Guide +Griffin measure module needs two configuration files to define the parameters of execution, one is for environment, the other is for dq job. + +## Environment Parameters +``` +{ + "spark": { + "log.level": "WARN", + "checkpoint.dir": "hdfs:///griffin/streaming/cp", + "batch.interval": "5s", + "process.interval": "30s", + "config": { + "spark.task.maxFailures": 5, + "spark.streaming.kafkaMaxRatePerPartition": 1000, + "spark.streaming.concurrentJobs": 4 + } + }, + + "persist": [ + { + "type": "log", + "config": { + "max.log.lines": 100 + } + }, { + "type": "hdfs", + "config": { + "path": "hdfs:///griffin/streaming/persist", + "max.persist.lines": 10000, + "max.lines.per.file": 10000 + } + } + ], + + "info.cache": [ + { + "type": "zk", + "config": { + "hosts": ":2181", + "namespace": "griffin/infocache", + "lock.path": "lock", + "mode": "persist", + "init.clear": true, + "close.clear": false + } + } + ] +} +``` +Above lists environment parameters. + +- **spark**: This field configures spark and spark streaming parameters. + + log.level: Level of spark log. + + checkpoint.dir: Check point directory of spark streaming, for streaming mode. + + 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. + + type: Metrics persist type, "log", "hdfs" or "http". + + config: Configure parameters of each persist type. + * log persist + - 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 + - api: api to submit persist metrics. + - method: http method, "post" default. +- **info.cache**: This field configures list of information cache parameters, multiple cache ways are supported. It is only for streaming dq case. + + type: Information cache type, "zk" for zookeeper cache. + + config: Configure parameters of info cache type. + * zookeeper cache + - hosts: zookeeper hosts list as a string, separated by comma. + - namespace: namespace of cache info, "" default. + - lock.path: path of lock info, "lock" default. + - mode: create mode of zookeeper node, "persist" default. + - init.clear: clear cache info when initialize, true default. + - close.clear: clear cache info when close connection, false default. + + +## DQ Job Parameters +``` +{ + "name": "accu_batch", + + "process.type": "batch", + + "data.sources": [ + { + "name": "src", + "connectors": [ + { + "type": "avro", + "version": "1.7", + "config": { + "file.name": "//.avro" + } + } + ] + }, { + "name": "tgt", + "connectors": [ + { + "type": "avro", + "version": "1.7", + "config": { + "file.name": "//.avro" + } + } + ] + } + ], + + "evaluateRule": { + "rules": [ + { + "dsl.type": "griffin-dsl", + "dq.type": "accuracy", + "rule": "src.user_id = tgt.user_id AND upper(src.first_name) = upper(tgt.first_name) AND src.last_name = tgt.last_name", + "details": { + "source": "src", + "target": "tgt", + "miss.records": { + "name": "miss.records", + "persist.type": "record" + }, + "accuracy": { + "name": "accu", + "persist.type": "metric" + }, + "miss": "miss_count", + "total": "total_count", + "matched": "matched_count" + } + } + ] + } +} +``` +Above lists DQ job configure parameters. + +- \ No newline at end of file From 3622c926644c4b2d2441067501e766578fa36f90 Mon Sep 17 00:00:00 2001 From: Lionel Liu Date: Wed, 11 Oct 2017 15:44:59 +0800 Subject: [PATCH 104/111] doc --- .gitignore | 2 + griffin-doc/measure-configuration-guide.md | 61 ++++++++++++++++++++++ 2 files changed, 63 insertions(+) diff --git a/.gitignore b/.gitignore index ad52fe537..97c2d5ee5 100644 --- a/.gitignore +++ b/.gitignore @@ -31,6 +31,8 @@ ui/bower_components/* ui/node_modules/* ui/debug.log ui/package-lock.json +ui/angular +ui/tmp derby.log metastore_db diff --git a/griffin-doc/measure-configuration-guide.md b/griffin-doc/measure-configuration-guide.md index 69cbcf704..8a0ec725f 100644 --- a/griffin-doc/measure-configuration-guide.md +++ b/griffin-doc/measure-configuration-guide.md @@ -75,3 +75,64 @@ Above lists environment parameters. ## DQ Job Parameters +``` +{ + "name": "accu_batch", + + "process.type": "batch", + + "data.sources": [ + { + "name": "src", + "connectors": [ + { + "type": "avro", + "version": "1.7", + "config": { + "file.name": "//.avro" + } + } + ] + }, { + "name": "tgt", + "connectors": [ + { + "type": "avro", + "version": "1.7", + "config": { + "file.name": "//.avro" + } + } + ] + } + ], + + "evaluateRule": { + "rules": [ + { + "dsl.type": "griffin-dsl", + "dq.type": "accuracy", + "rule": "src.user_id = tgt.user_id AND upper(src.first_name) = upper(tgt.first_name) AND src.last_name = tgt.last_name", + "details": { + "source": "src", + "target": "tgt", + "miss.records": { + "name": "miss.records", + "persist.type": "record" + }, + "accuracy": { + "name": "accu", + "persist.type": "metric" + }, + "miss": "miss_count", + "total": "total_count", + "matched": "matched_count" + } + } + ] + } +} +``` +Above lists DQ job configure parameters. + +- \ No newline at end of file From 15995970fb7d753e59ff914ed7f28edacb6eeeb4 Mon Sep 17 00:00:00 2001 From: Lionel Liu Date: Wed, 11 Oct 2017 15:48:51 +0800 Subject: [PATCH 105/111] reset --- .gitignore | 1 - 1 file changed, 1 deletion(-) diff --git a/.gitignore b/.gitignore index 97c2d5ee5..ee478d874 100644 --- a/.gitignore +++ b/.gitignore @@ -31,7 +31,6 @@ ui/bower_components/* ui/node_modules/* ui/debug.log ui/package-lock.json -ui/angular ui/tmp derby.log From 5ed121177ddf7709527ae9250835ddaae6ee20ab Mon Sep 17 00:00:00 2001 From: Lionel Liu Date: Wed, 11 Oct 2017 21:35:15 +0800 Subject: [PATCH 106/111] text dir batch data connector dir depth bug fix --- griffin-doc/measure-configuration-guide.md | 98 ++++++++++++++----- .../batch/TextDirBatchDataConnector.scala | 2 +- 2 files changed, 74 insertions(+), 26 deletions(-) diff --git a/griffin-doc/measure-configuration-guide.md b/griffin-doc/measure-configuration-guide.md index 9337831a7..cc4bde0d1 100644 --- a/griffin-doc/measure-configuration-guide.md +++ b/griffin-doc/measure-configuration-guide.md @@ -74,29 +74,32 @@ 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. - + type: Metrics persist type, "log", "hdfs" or "http". - + config: Configure parameters of each persist type. - * log persist - - 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 - - api: api to submit persist metrics. - - method: http method, "post" default. -- **info.cache**: This field configures list of information cache parameters, multiple cache ways are supported. It is only for streaming dq case. - + type: Information cache type, "zk" for zookeeper cache. - + config: Configure parameters of info cache type. - * zookeeper cache - - hosts: zookeeper hosts list as a string, separated by comma. - - namespace: namespace of cache info, "" default. - - lock.path: path of lock info, "lock" default. - - mode: create mode of zookeeper node, "persist" default. - - init.clear: clear cache info when initialize, true default. - - close.clear: clear cache info when close connection, false default. +- **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). +### Persist +- **type**: Metrics persist type, "log", "hdfs" and "http". +- **config**: Configure parameters of each persist type. + + log persist + * 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 + * api: api to submit persist metrics. + * method: http method, "post" default. + +### Info Cache +- **type**: Information cache type, "zk" for zookeeper cache. +- **config**: Configure parameters of info cache type. + + zookeeper cache + * 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. + * mode: create mode of zookeeper node, "persist" as default. + * init.clear: clear cache info when initialize, true default. + * close.clear: clear cache info when close connection, false default. ## DQ Job Parameters ``` @@ -113,7 +116,8 @@ Above lists environment parameters. "type": "avro", "version": "1.7", "config": { - "file.name": "//.avro" + "file.path": "/", + "file.name": ".avro" } } ] @@ -124,7 +128,8 @@ Above lists environment parameters. "type": "avro", "version": "1.7", "config": { - "file.name": "//.avro" + "file.path": "/", + "file.name": ".avro" } } ] @@ -159,4 +164,47 @@ Above lists environment parameters. ``` Above lists DQ job configure parameters. -- \ No newline at end of file +- **name**: Name of DQ job. +- **process.type**: Process type of DQ job, "batch" or "streaming". +- **data.sources**: List of data sources in this DQ job. + + name: Name of this data source, it should be different from other data sources. + + connectors: List of data connectors combined as the same data source. Details of data connector configuration [here](#data-connector). +- **evaluateRule**: Evaluate rule parameters of this DQ job. + + dsl.type: Default dsl type of all the rules. + + rules: List of rules, to define every rule step. Details of rule configuration [here](#rule). + +### Data Connector +- **type**: Data connector type, "avro", "hive", "text-dir" for batch mode, "kafka" for streaming mode. +- **version**: Version string of data connector type. +- **config**: Configure parameters of each data connector type. + + avro data connector + * file.path: avro file path, optional, "" as default. + * file.name: avro file name. + + hive data connector + * database: data base name, optional, "default" as default. + * table.name: table name. + * partitions: partition conditions string, split by ";" and ",", optional. + e.g. `dt=20170410, hour=15; dt=20170411, hour=15; dt=20170412, hour=15` + + text dir data connector + * dir.path: parent directory path. + * data.dir.depth: integer, depth of data directories, 0 as default. + * success.file: success file name, + * done.file: + +### Rule +- **dsl.type**: Rule dsl type, "spark-sql", "df-opr" and "griffin-dsl". +- **name** (step information): Result table name of this rule, optional for "griffin-dsl" type. +- **persist.type** (step information): Persist type of result table, optional for "griffin-dsl" type. Supporting "metric", "record" and "none" type, "metric" type indicates the result will be persisted as metrics, "record" type indicates the result will be persisted as record only, "none" type indicates the result will not be persisted. Default is "none" type. +- **update.data.source** (step information): If the result table needs to update the data source, this parameter is the data source name, for streaming accuracy case, optional. +- **dq.type**: DQ type of this rule, only for "griffin-dsl" type, supporting "accuracy" and "profiling". +- **details**: Details of this rule, optional. + + accuracy dq type detail configuration + * source: the data source name which as source in accuracy, default is the name of first data source in "data.sources" if not configured. + * target: the data source name which as target in accuracy, default is the name of second data source in "data.sources" if not configured. + * miss.records: step information of miss records result table step in accuracy. + * accuracy: step information of accuracy result table step in accuracy. + * miss: alias of miss column in result table. + * total: alias of total column in result table. + * matched: alias of matched column in result table. + + profiling dq type detail configuration + * profiling: step information of profiling result table step in profiling. \ No newline at end of file diff --git a/measure/src/main/scala/org/apache/griffin/measure/data/connector/batch/TextDirBatchDataConnector.scala b/measure/src/main/scala/org/apache/griffin/measure/data/connector/batch/TextDirBatchDataConnector.scala index abc547bc3..13ffe8979 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/data/connector/batch/TextDirBatchDataConnector.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/data/connector/batch/TextDirBatchDataConnector.scala @@ -75,7 +75,7 @@ case class TextDirBatchDataConnector(sqlContext: SQLContext, dqEngines: DqEngine if (depth <= 0) { subDirs.filter(filteFunc) } else { - listSubDirs(subDirs, depth, filteFunc) + listSubDirs(subDirs, depth - 1, filteFunc) } } From 3ffdacb92cb955c4171fac972229e63523fd9ca4 Mon Sep 17 00:00:00 2001 From: Lionel Liu Date: Wed, 11 Oct 2017 21:37:22 +0800 Subject: [PATCH 107/111] doc --- griffin-doc/measure-batch-sample.md | 19 +++++++++++++++++++ griffin-doc/measure-streaming-sample.md | 1 + 2 files changed, 20 insertions(+) create mode 100644 griffin-doc/measure-batch-sample.md diff --git a/griffin-doc/measure-batch-sample.md b/griffin-doc/measure-batch-sample.md new file mode 100644 index 000000000..3c82fb9a3 --- /dev/null +++ b/griffin-doc/measure-batch-sample.md @@ -0,0 +1,19 @@ + + diff --git a/griffin-doc/measure-streaming-sample.md b/griffin-doc/measure-streaming-sample.md index 004ed3ba4..2dc69c145 100644 --- a/griffin-doc/measure-streaming-sample.md +++ b/griffin-doc/measure-streaming-sample.md @@ -16,6 +16,7 @@ KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License. --> + # Measure streaming sample Measures consists of batch measure and streaming measure. This document is for the streaming measure sample. From 5db092c4568b2a704dfacade5bdd7bb972bf10c1 Mon Sep 17 00:00:00 2001 From: Lionel Liu Date: Wed, 11 Oct 2017 21:42:56 +0800 Subject: [PATCH 108/111] doc --- griffin-doc/measure-batch-sample.md | 9 +++++++++ ...reaming-sample.md => measure-streaming-sample-old.md} | 1 - 2 files changed, 9 insertions(+), 1 deletion(-) rename griffin-doc/{measure-streaming-sample.md => measure-streaming-sample-old.md} (99%) diff --git a/griffin-doc/measure-batch-sample.md b/griffin-doc/measure-batch-sample.md index 3c82fb9a3..e059aef56 100644 --- a/griffin-doc/measure-batch-sample.md +++ b/griffin-doc/measure-batch-sample.md @@ -17,3 +17,12 @@ specific language governing permissions and limitations under the License. --> +# Measure Batch Sample +Measures consists of batch measure and streaming measure. This document is for the batch measure sample. + +### Data source +At current, we support avro, hive and text-dir as batch data source. +In this sample, we will use avro file as data source. + +### Measure type +At current, we support accuracy measure in streaming mode. \ No newline at end of file diff --git a/griffin-doc/measure-streaming-sample.md b/griffin-doc/measure-streaming-sample-old.md similarity index 99% rename from griffin-doc/measure-streaming-sample.md rename to griffin-doc/measure-streaming-sample-old.md index 2dc69c145..004ed3ba4 100644 --- a/griffin-doc/measure-streaming-sample.md +++ b/griffin-doc/measure-streaming-sample-old.md @@ -16,7 +16,6 @@ KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License. --> - # Measure streaming sample Measures consists of batch measure and streaming measure. This document is for the streaming measure sample. From e2450f2d3ee81506691fcf4719709b15409912fd Mon Sep 17 00:00:00 2001 From: Lionel Liu Date: Wed, 11 Oct 2017 22:18:49 +0800 Subject: [PATCH 109/111] measure batch sample --- griffin-doc/measure-batch-sample.md | 120 ++++++++++++++++++++- griffin-doc/measure-configuration-guide.md | 1 + 2 files changed, 117 insertions(+), 4 deletions(-) diff --git a/griffin-doc/measure-batch-sample.md b/griffin-doc/measure-batch-sample.md index e059aef56..3783f9477 100644 --- a/griffin-doc/measure-batch-sample.md +++ b/griffin-doc/measure-batch-sample.md @@ -20,9 +20,121 @@ under the License. # Measure Batch Sample Measures consists of batch measure and streaming measure. This document is for the batch measure sample. +## Batch Accuracy Sample +``` +{ + "name": "accu_batch", + + "process.type": "batch", + + "data.sources": [ + { + "name": "src", + "connectors": [ + { + "type": "avro", + "version": "1.7", + "config": { + "file.name": "users_info_src.avro" + } + } + ] + }, { + "name": "tgt", + "connectors": [ + { + "type": "avro", + "version": "1.7", + "config": { + "file.name": "users_info_target.avro" + } + } + ] + } + ], + + "evaluateRule": { + "rules": [ + { + "dsl.type": "griffin-dsl", + "dq.type": "accuracy", + "rule": "src.user_id = tgt.user_id AND upper(src.first_name) = upper(tgt.first_name) AND src.last_name = tgt.last_name", + "details": { + "source": "src", + "target": "tgt", + "miss.records": { + "name": "miss.records", + "persist.type": "record" + }, + "accuracy": { + "name": "accu", + "persist.type": "metric" + }, + "miss": "miss_count", + "total": "total_count", + "matched": "matched_count" + } + } + ] + } +} +``` +Above is the configure file of batch accuracy job. + +### Data source +In this sample, we use avro file as source and target. + +### Evaluate rule +In this accuracy sample, the rule describes the match condition: `src.user_id = tgt.user_id AND upper(src.first_name) = upper(tgt.first_name) AND src.last_name = tgt.last_name`. +The accuracy metrics will be persisted as metric, with miss column named "miss_count", total column named "total_count", matched column named "matched_count". +The miss records of source will be persisted as record. + +## Batch Profiling Sample +``` +{ + "name": "prof_batch_test", + + "process.type": "batch", + + "data.sources": [ + { + "name": "source", + "connectors": [ + { + "type": "hive", + "version": "1.2", + "config": { + "database": "griffin", + "table.name": "demo_src" + } + } + ] + } + ], + + "evaluateRule": { + "rules": [ + { + "dsl.type": "griffin-dsl", + "dq.type": "profiling", + "rule": "country, country.count() as cnt group by country order by cnt desc limit 3", + "details": { + "source": "source", + "profiling": { + "name": "cntry-group", + "persist.type": "metric" + } + } + } + ] + } +} +``` +Above is the configure file of batch profiling job. + ### Data source -At current, we support avro, hive and text-dir as batch data source. -In this sample, we will use avro file as data source. +In this sample, we use hive table as source. -### Measure type -At current, we support accuracy measure in streaming mode. \ No newline at end of file +### Evaluate rule +In this profiling sample, the rule describes the profiling request: `country, country.count() as cnt group by country order by cnt desc limit 3`. +The profiling metrics will be persisted as metric, listing the most 3 groups of items in same country. \ No newline at end of file diff --git a/griffin-doc/measure-configuration-guide.md b/griffin-doc/measure-configuration-guide.md index cc4bde0d1..06329276d 100644 --- a/griffin-doc/measure-configuration-guide.md +++ b/griffin-doc/measure-configuration-guide.md @@ -207,4 +207,5 @@ Above lists DQ job configure parameters. * total: alias of total column in result table. * matched: alias of matched column in result table. + 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. * profiling: step information of profiling result table step in profiling. \ No newline at end of file From 35d7d76b6e87a78da7df81b2d6d961ba7a7786dc Mon Sep 17 00:00:00 2001 From: Lionel Liu Date: Thu, 12 Oct 2017 12:12:00 +0800 Subject: [PATCH 110/111] hive batch partition bug fix --- .../batch/HiveBatchDataConnector.scala | 32 +++++++++++++------ .../data/connector/ConnectorTest.scala | 26 +++++++++++++++ 2 files changed, 49 insertions(+), 9 deletions(-) diff --git a/measure/src/main/scala/org/apache/griffin/measure/data/connector/batch/HiveBatchDataConnector.scala b/measure/src/main/scala/org/apache/griffin/measure/data/connector/batch/HiveBatchDataConnector.scala index 5d80d0e5f..20c9e2488 100644 --- a/measure/src/main/scala/org/apache/griffin/measure/data/connector/batch/HiveBatchDataConnector.scala +++ b/measure/src/main/scala/org/apache/griffin/measure/data/connector/batch/HiveBatchDataConnector.scala @@ -48,17 +48,28 @@ case class HiveBatchDataConnector(sqlContext: SQLContext, dqEngines: DqEngines, val partitionsString = config.getString(Partitions, "") val concreteTableName = s"${database}.${tableName}" - val partitions = partitionsString.split(";").map(s => s.split(",").map(_.trim)) +// val partitions = partitionsString.split(";").map(s => s.split(",").map(_.trim)) + val partitions: Array[Array[String]] = partitionsString.split(";").flatMap { s => + val arr = s.trim.split(",").flatMap { t => + t.trim match { + case p if (p.nonEmpty) => Some(p) + case _ => None + } + } + if (arr.size > 0) Some(arr) else None + } def data(ms: Long): Option[DataFrame] = { try { - val df = sqlContext.sql(dataSql) + val dtSql = dataSql + info(dtSql) + val df = sqlContext.sql(dtSql) val dfOpt = Some(df) val preDfOpt = preProcess(dfOpt, ms) preDfOpt } catch { case e: Throwable => { - error(s"load hive table ${concreteTableName} fails") + error(s"load hive table ${concreteTableName} fails: ${e.getMessage}") None } } @@ -131,12 +142,15 @@ case class HiveBatchDataConnector(sqlContext: SQLContext, dqEngines: DqEngines, } private def dataSql(): String = { - val clauses = partitions.map { prtn => - val cls = prtn.mkString(" AND ") - if (cls.isEmpty) s"SELECT * FROM ${concreteTableName}" - else s"SELECT * FROM ${concreteTableName} WHERE ${cls}" - } - clauses.mkString(" UNION ALL ") + val tableClause = s"SELECT * FROM ${concreteTableName}" + val validPartitions = partitions.filter(_.size > 0) + if (validPartitions.size > 0) { + val clauses = validPartitions.map { prtn => + val cls = prtn.mkString(" AND ") + s"${tableClause} WHERE ${cls}" + } + clauses.mkString(" UNION ALL ") + } else tableClause } // private def toTuple[A <: AnyRef](as: Seq[A]): Product = { diff --git a/measure/src/test/scala/org/apache/griffin/measure/data/connector/ConnectorTest.scala b/measure/src/test/scala/org/apache/griffin/measure/data/connector/ConnectorTest.scala index ead84f735..30cacc4b1 100644 --- a/measure/src/test/scala/org/apache/griffin/measure/data/connector/ConnectorTest.scala +++ b/measure/src/test/scala/org/apache/griffin/measure/data/connector/ConnectorTest.scala @@ -67,5 +67,31 @@ class ConnectorTest extends FunSuite with Matchers with BeforeAndAfter { val clazz = Class.forName(tp) ClassTag(clazz) } + + test ("test sql") { + val concreteTableName = "default.demo_src" + val partitionsString = " dt=1, hr=2; dt=1 ; ;,hr=4; dt=2 ;;dt=5" + val partitions: Array[Array[String]] = partitionsString.split(";").flatMap { s => + val arr = s.trim.split(",").flatMap { t => + t.trim match { + case p if (p.nonEmpty) => Some(p) + case _ => None + } + } + if (arr.size > 0) Some(arr) else None + } + + val tableClause = s"SELECT * FROM ${concreteTableName}" + val validPartitions = partitions.filter(_.size > 0) + val ret = if (validPartitions.size > 0) { + val clauses = validPartitions.map { prtn => + val cls = prtn.mkString(" AND ") + s"${tableClause} WHERE ${cls}" + } + clauses.mkString(" UNION ALL ") + } else tableClause + + println(ret) + } } From 57c9789746caff45e4dae286fbe4b69fab702505 Mon Sep 17 00:00:00 2001 From: Lionel Liu Date: Thu, 12 Oct 2017 12:25:12 +0800 Subject: [PATCH 111/111] docker doc --- griffin-doc/measure-demo-docker.md | 63 ++++++++++++++++++++++++++++++ 1 file changed, 63 insertions(+) create mode 100644 griffin-doc/measure-demo-docker.md diff --git a/griffin-doc/measure-demo-docker.md b/griffin-doc/measure-demo-docker.md new file mode 100644 index 000000000..49040dac8 --- /dev/null +++ b/griffin-doc/measure-demo-docker.md @@ -0,0 +1,63 @@ + + +# Griffin Measure Demo Docker +We've prepared a docker for griffin measure demo. + +## Preparation +1. Install docker. +2. Download docker image. In this image, the environment for measure module has been prepared, including: hadoop, hive, spark, mysql. +``` +docker pull bhlx3lyx7/griffin_measure_demo:0.0.1 +``` +3. Run docker image. +``` +docker run -it -h griffin --name griffin_measure_demo -m 8G --memory-swap -1 \ +-p 42122:2122 -p 47077:7077 -p 46066:6066 -p 48088:8088 -p 48040:8040 \ +-p 43306:3306 -p 49000:9000 -p 48042:8042 -p 48080:8080 -p 47017:27017 \ +-p 49083:9083 -p 48998:8998 -p 49200:9200 bhlx3lyx7/griffin_measure_demo:0.0.1 +``` +4. In this docker container, run the prepared demo. +- **accuracy demo**: This demo is batch accuracy, source data is Hive table "demo_src", target data is Hive table "demo_tgt", metrics will be persisted in `hdfs:///griffin/persist/accu` after calculation. + + switch into `job/accu`. + ``` + cd job/accu + ``` + + run the prepared script. + ``` + ./bgwork.sh + ``` + + check job log. + ``` + tail -f accu.log + ``` +- **profiling demo**: This demo is batch profiling, source data is Hive table "demo_src", metrics will be persisted in `hdfs:///griffin/persist/prof` after calculation. + + switch into `job/prof`. + ``` + cd job/prof + ``` + + run the prepared script. + ``` + ./bgwork.sh + ``` + + check job log. + ``` + tail -f prof.log + ``` +5. You can modify the job configuration file `config.json` of the above demos, or create your own data sources, to get more metrics of data. \ No newline at end of file