You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@griffin.apache.org by gu...@apache.org on 2017/05/04 03:04:10 UTC

[02/51] [partial] incubator-griffin git commit: refactor arch

http://git-wip-us.apache.org/repos/asf/incubator-griffin/blob/f629d0f4/service/src/main/java/org/apache/griffin/core/schedule/SchedulerController.java
----------------------------------------------------------------------
diff --git a/service/src/main/java/org/apache/griffin/core/schedule/SchedulerController.java b/service/src/main/java/org/apache/griffin/core/schedule/SchedulerController.java
new file mode 100644
index 0000000..84b1da0
--- /dev/null
+++ b/service/src/main/java/org/apache/griffin/core/schedule/SchedulerController.java
@@ -0,0 +1,156 @@
+package org.apache.griffin.core.schedule;
+
+import org.quartz.*;
+import org.quartz.impl.matchers.GroupMatcher;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.springframework.beans.factory.annotation.Autowired;
+import org.springframework.scheduling.quartz.SchedulerFactoryBean;
+import org.springframework.web.bind.annotation.*;
+
+import javax.ws.rs.Produces;
+import javax.ws.rs.core.MediaType;
+import java.io.Serializable;
+import java.text.ParseException;
+import java.util.*;
+
+import static org.quartz.DateBuilder.futureDate;
+import static org.quartz.JobBuilder.newJob;
+import static org.quartz.JobKey.jobKey;
+import static org.quartz.TriggerBuilder.newTrigger;
+import static org.quartz.TriggerKey.triggerKey;
+
+@RestController
+@RequestMapping("/jobs")
+public class SchedulerController {
+    private static final Logger LOGGER = LoggerFactory.getLogger(SchedulerController.class);
+
+    @Autowired
+    private SchedulerFactoryBean factory;
+
+
+    public static final String ACCURACY_BATCH_GROUP = "BA";
+
+    @RequestMapping("/")
+    public List<Map<String, Serializable>> jobs() throws SchedulerException,
+            ParseException {
+        Scheduler scheduler = factory.getObject();
+
+        List<Map<String, Serializable>> list = new ArrayList<>();
+        for (String groupName : scheduler.getJobGroupNames()) {
+            for (JobKey jobKey : scheduler.getJobKeys(GroupMatcher
+                    .jobGroupEquals(groupName))) {
+                String jobName = jobKey.getName();
+                String jobGroup = jobKey.getGroup();
+
+                JobDetail jd = scheduler.getJobDetail(jobKey);
+
+                List<Trigger> triggers = (List<Trigger>) scheduler
+                        .getTriggersOfJob(jobKey);
+                Map<String, Serializable> map = new HashMap<>();
+
+                if (triggers.size() > 0) {
+                    //always get next run
+                    Date nextFireTime = triggers.get(0).getNextFireTime();
+
+                    map.put("jobName", jobName);
+                    map.put("groupName", jobGroup);
+                    map.put("nextFireTime", nextFireTime.toString());
+
+                    map.put("measure", (String) jd.getJobDataMap().get("measure"));
+                    list.add(map);
+                }
+
+            }
+        }
+        return list;
+    }
+
+    @RequestMapping(value = "/{groupName}/{jobName}", method = RequestMethod.DELETE)
+    public String removeTask(@PathVariable String groupName, @PathVariable String jobName)
+            throws SchedulerException {
+        Scheduler scheduler = factory.getObject();
+        JobKey jobKey = new JobKey(jobName, groupName);
+        scheduler.deleteJob(jobKey);
+        return "task has been removed";
+    }
+
+    //@RequestMapping(value = "/add/{groupName}/{jobName}/{measureName}/{sourcePat}/{targetPat}")
+    @RequestMapping(value = "/add/{groupName}/{jobName}/{measureName}", method = RequestMethod.POST)
+    @ResponseBody
+    @Produces(MediaType.APPLICATION_JSON)
+    public Boolean addTask(@PathVariable String groupName,
+                           @PathVariable String jobName,
+                           @PathVariable String measureName,
+                           @RequestBody SchedulerRequestBody schedulerRequestBody) {
+        //@RequestParam(value = "sourcePat", required = false) String sourcePat,
+        //@RequestParam(value = "targetPat", required = false) String targetPat
+        int jobStartTime = 0;
+        int periodTime = 0;
+        try {
+            Scheduler scheduler = factory.getObject();
+            TriggerKey triggerKey = triggerKey(jobName, groupName);
+
+            if (scheduler.checkExists(triggerKey)) {
+                scheduler.unscheduleJob(triggerKey);
+            }
+            JobKey jobKey = jobKey(jobName, groupName);
+            JobDetail jobDetail;
+            if (scheduler.checkExists(jobKey)) {
+                jobDetail = scheduler.getJobDetail(jobKey);
+                scheduler.addJob(jobDetail, true);
+            } else {
+                jobDetail = newJob(SparkSubmitJob.class)
+                        .storeDurably()
+                        .withIdentity(jobKey)
+                        .build();
+
+                jobDetail.getJobDataMap().put("measure", measureName);
+                jobDetail.getJobDataMap().put("sourcePat", schedulerRequestBody.getSourcePat());
+                jobDetail.getJobDataMap().put("targetPat", schedulerRequestBody.getTargetPat());
+                jobDetail.getJobDataMap().put("dataStartTimestamp", schedulerRequestBody.getDataStartTimestamp());
+                jobDetail.getJobDataMap().put("jobStartTime", schedulerRequestBody.getJobStartTime());
+                jobDetail.getJobDataMap().put("periodTime", schedulerRequestBody.getPeriodTime());
+                jobDetail.getJobDataMap().put("lastTime", "");
+
+                jobStartTime = Integer.parseInt(schedulerRequestBody.getJobStartTime());
+                periodTime = Integer.parseInt(schedulerRequestBody.getPeriodTime());
+                scheduler.addJob(jobDetail, false);
+            }
+
+            Trigger trigger = newTrigger()
+                    .withIdentity(triggerKey)
+                    .forJob(jobDetail)
+                    //hardcode to
+//					.withSchedule(CronScheduleBuilder.cronSchedule("0 0/1 0 * * ?"))
+                    .withSchedule(SimpleScheduleBuilder.simpleSchedule()
+                            .withIntervalInSeconds(periodTime)
+                            .repeatForever())
+                    .startAt(futureDate(jobStartTime, DateBuilder.IntervalUnit.SECOND))
+                    .build();
+
+            scheduler.scheduleJob(trigger);
+
+            return true;
+
+        } catch (SchedulerException e) {
+            LOGGER.error("", e);
+            return false;
+        }
+    }
+
+
+    @RequestMapping(value = "/groups/{group}/jobs/{name}", method = RequestMethod.DELETE)
+    public boolean deleteJob(@PathVariable String group, @PathVariable String name) {
+        try {
+            Scheduler scheduler = factory.getObject();
+            scheduler.deleteJob(new JobKey(name, group));
+            return true;
+        } catch (SchedulerException e) {
+            LOGGER.error(e.getMessage());
+            return false;
+        }
+    }
+}
+
+

http://git-wip-us.apache.org/repos/asf/incubator-griffin/blob/f629d0f4/service/src/main/java/org/apache/griffin/core/schedule/SchedulerRequestBody.java
----------------------------------------------------------------------
diff --git a/service/src/main/java/org/apache/griffin/core/schedule/SchedulerRequestBody.java b/service/src/main/java/org/apache/griffin/core/schedule/SchedulerRequestBody.java
new file mode 100644
index 0000000..354bf08
--- /dev/null
+++ b/service/src/main/java/org/apache/griffin/core/schedule/SchedulerRequestBody.java
@@ -0,0 +1,53 @@
+package org.apache.griffin.core.schedule;
+
+/**
+ * Created by xiangrchen on 4/27/17.
+ */
+public class SchedulerRequestBody {
+    String sourcePat;
+    String targetPat;
+    String dataStartTimestamp;
+    String jobStartTime;
+    String periodTime;
+
+    public String getSourcePat() {
+        return sourcePat;
+    }
+
+    public void setSourcePat(String sourcePat) {
+        this.sourcePat = sourcePat;
+    }
+
+    public String getTargetPat() {
+        return targetPat;
+    }
+
+    public void setTargetPat(String targetPat) {
+        this.targetPat = targetPat;
+    }
+
+    public String getDataStartTimestamp() {
+        return dataStartTimestamp;
+    }
+
+    public void setDataStartTimestamp(String dataStartTimestamp) {
+        this.dataStartTimestamp = dataStartTimestamp;
+    }
+
+    public String getJobStartTime() {
+        return jobStartTime;
+    }
+
+    public void setJobStartTime(String jobStartTime) {
+        this.jobStartTime = jobStartTime;
+    }
+
+    public String getPeriodTime() {
+        return periodTime;
+    }
+
+    public void setPeriodTime(String periodTime) {
+        this.periodTime = periodTime;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-griffin/blob/f629d0f4/service/src/main/java/org/apache/griffin/core/schedule/SparkJobDO.java
----------------------------------------------------------------------
diff --git a/service/src/main/java/org/apache/griffin/core/schedule/SparkJobDO.java b/service/src/main/java/org/apache/griffin/core/schedule/SparkJobDO.java
new file mode 100644
index 0000000..65f7f41
--- /dev/null
+++ b/service/src/main/java/org/apache/griffin/core/schedule/SparkJobDO.java
@@ -0,0 +1,131 @@
+package org.apache.griffin.core.schedule;
+
+import java.io.Serializable;
+import java.util.List;
+
+/**
+ * Created by xiangrchen on 4/26/17.
+ */
+public class SparkJobDO implements Serializable{
+
+    String file;
+
+    String className;
+
+    List<String> args;
+
+    String name;
+
+    String queue;
+
+    Long numExecutors;
+
+    Long executorCores;
+
+    String driverMemory;
+
+    String executorMemory;
+
+    Conf conf;
+
+    List<String> jars;
+
+    List<String> files;
+
+    public String getFile() {
+        return file;
+    }
+
+    public void setFile(String file) {
+        this.file = file;
+    }
+
+    public String getClassName() {
+        return className;
+    }
+
+    public void setClassName(String className) {
+        this.className = className;
+    }
+
+    public List<String> getArgs() {
+        return args;
+    }
+
+    public void setArgs(List<String> args) {
+        this.args = args;
+    }
+
+    public String getName() {
+        return name;
+    }
+
+    public void setName(String name) {
+        this.name = name;
+    }
+
+    public String getQueue() {
+        return queue;
+    }
+
+    public void setQueue(String queue) {
+        this.queue = queue;
+    }
+
+    public Long getNumExecutors() {
+        return numExecutors;
+    }
+
+    public void setNumExecutors(Long numExecutors) {
+        this.numExecutors = numExecutors;
+    }
+
+    public Long getExecutorCores() {
+        return executorCores;
+    }
+
+    public void setExecutorCores(Long executorCores) {
+        this.executorCores = executorCores;
+    }
+
+    public String getDriverMemory() {
+        return driverMemory;
+    }
+
+    public void setDriverMemory(String driverMemory) {
+        this.driverMemory = driverMemory;
+    }
+
+    public String getExecutorMemory() {
+        return executorMemory;
+    }
+
+    public void setExecutorMemory(String executorMemory) {
+        this.executorMemory = executorMemory;
+    }
+
+    public Conf getConf() {
+        return conf;
+    }
+
+    public void setConf(Conf conf) {
+        this.conf = conf;
+    }
+
+    public List<String> getJars() {
+        return jars;
+    }
+
+    public void setJars(List<String> jars) {
+        this.jars = jars;
+    }
+
+    public List<String> getFiles() {
+        return files;
+    }
+
+    public void setFiles(List<String> files) {
+        this.files = files;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-griffin/blob/f629d0f4/service/src/main/java/org/apache/griffin/core/schedule/SparkSubmitJob.java
----------------------------------------------------------------------
diff --git a/service/src/main/java/org/apache/griffin/core/schedule/SparkSubmitJob.java b/service/src/main/java/org/apache/griffin/core/schedule/SparkSubmitJob.java
new file mode 100644
index 0000000..cc8e1c5
--- /dev/null
+++ b/service/src/main/java/org/apache/griffin/core/schedule/SparkSubmitJob.java
@@ -0,0 +1,221 @@
+package org.apache.griffin.core.schedule;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.ObjectWriter;
+import org.apache.griffin.core.measure.DataConnector;
+import org.apache.griffin.core.measure.Measure;
+import org.apache.griffin.core.measure.repo.MeasureRepo;
+import org.quartz.*;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.springframework.beans.factory.annotation.Autowired;
+import org.springframework.web.client.RestTemplate;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.text.SimpleDateFormat;
+import java.util.*;
+
+@PersistJobDataAfterExecution
+@DisallowConcurrentExecution
+public class SparkSubmitJob implements Job {
+    Logger logger = LoggerFactory.getLogger(getClass());
+
+    @Autowired
+    MeasureRepo measureRepo;
+
+    Properties props;
+
+    String patItem;
+    /**
+     * partitionItemSet
+     * for example
+     * partitionItemSet like "date","hour",...
+     */
+    String[] partitionItemSet;
+    /**
+     * sourcePatternItemSet targetPatternItemSet
+     * for example
+     * sourcePatternItemSet like "YYYYMMDD","HH",...
+     */
+    String[] sourcePatternItemSet;
+    String[] targetPatternItemSet;
+    /**
+     * sourcePatternPartitionSizeMin is the min between partitionItemSet.length and sourcePatternItemSet.length
+     */
+    int sourcePatternPartitionSizeMin = 0;
+    int targetPatternPartitionSizeMin = 0;
+
+    Measure measure;
+    String sourcePattern;
+    String targetPattern;
+    String dataStartTimestamp;
+    String eachJoblastTimestamp;
+    String periodTime;
+
+    String uri;
+    SparkJobDO sparkJobDO = new SparkJobDO();
+
+    public SparkSubmitJob() throws IOException {
+        try {
+            props = getsparkJobProperties();
+            patItem = props.getProperty("sparkJob.dateAndHour");
+            partitionItemSet = patItem.split(",");
+            uri = props.getProperty("sparkJob.uri");
+        } catch (IOException e) {
+            e.printStackTrace();
+        }
+    }
+
+    @Override
+    public void execute(JobExecutionContext context) {
+        JobDetail jd = context.getJobDetail();
+        String measureName = jd.getJobDataMap().getString("measure");
+        measure = measureRepo.findByName(measureName);
+        sourcePattern = jd.getJobDataMap().getString("sourcePat");
+        targetPattern = jd.getJobDataMap().getString("targetPat");
+        dataStartTimestamp = jd.getJobDataMap().getString("dataStartTimestamp");
+        eachJoblastTimestamp = jd.getJobDataMap().getString("lastTime");
+        logger.info(eachJoblastTimestamp);
+        periodTime = jd.getJobDataMap().getString("periodTime");
+        //prepare current system timestamp
+        long currentSystemTimestamp = System.currentTimeMillis();
+
+        if (sourcePattern != null && !sourcePattern.isEmpty()) {
+            sourcePatternItemSet = sourcePattern.split("-");
+            sourcePatternPartitionSizeMin = Math.min(partitionItemSet.length, sourcePatternItemSet.length);
+            long currentTimstamp = setCurrentTimestamp(currentSystemTimestamp);
+            setDataConnectorPartitions(measure.getSource(), sourcePatternItemSet, partitionItemSet, currentTimstamp, sourcePatternPartitionSizeMin);
+            jd.getJobDataMap().put("lastTime", currentTimstamp + "");
+        }
+        if (targetPattern != null && !targetPattern.equals("")) {
+            targetPatternItemSet = targetPattern.split("-");
+            targetPatternPartitionSizeMin = Math.min(partitionItemSet.length, targetPatternItemSet.length);
+            long currentTimstamp = setCurrentTimestamp(currentSystemTimestamp);
+            setDataConnectorPartitions(measure.getTarget(), targetPatternItemSet, partitionItemSet, currentTimstamp, targetPatternPartitionSizeMin);
+            jd.getJobDataMap().put("lastTime", currentTimstamp + "");
+        }
+        //final String uri = "http://10.9.246.187:8998/batches";
+        RestTemplate restTemplate = new RestTemplate();
+        setSparkJobDO();
+
+        String result = restTemplate.postForObject(uri, sparkJobDO, String.class);
+        logger.info(result);
+        //	{"file": "/exe/griffin-measure-batch-0.0.1-SNAPSHOT.jar", "className": "org.apache.griffin.measure.batch.Application", "args": ["/benchmark/test/env.json", "/benchmark/test/config-rdm.json"], "name": "griffin-livy", "queue": "default", "numExecutors": 2, "executorCores": 4, "driverMemory": "2g", "executorMemory": "2g", "conf": {"spark.jars.packages": "com.databricks:spark-avro_2.10:2.0.1"}, "jars": ["/livy/datanucleus-api-jdo-3.2.6.jar", "/livy/datanucleus-core-3.2.10.jar", "/livy/datanucleus-rdbms-3.2.9.jar"], "files": ["/livy/hive-site.xml"]}' -H "Content-Type: application/json"
+    }
+
+    public Map<String, String> genPartitions(String[] PatternItemSet, String[] partitionItemSet, long timestamp, int comparableSizeMin) {
+        Map<String, String> res = new HashMap<>();
+        for (int i = 0; i < comparableSizeMin; i++) {
+            /**
+             * in order to get a standard date like 20170427 01
+             */
+            String pattrn = PatternItemSet[i].replace("mm", "MM");
+            pattrn = pattrn.replace("DD", "dd");
+            pattrn = pattrn.replace("hh", "HH");
+            SimpleDateFormat sdf = new SimpleDateFormat(pattrn);
+            res.put(partitionItemSet[i], sdf.format(new Date(timestamp)));
+        }
+        return res;
+    }
+
+    /**
+     * putDataConnectorPartitions
+     *
+     * @param dc
+     * @param patternItemSet
+     * @param partitionItemSet
+     * @param timestamp
+     * @param patternPartitionSizeMin
+     * @return
+     */
+    private void setDataConnectorPartitions(DataConnector dc, String[] patternItemSet, String[] partitionItemSet, long timestamp, int patternPartitionSizeMin) {
+        Map<String, String> partitionItemMap = genPartitions(patternItemSet, partitionItemSet, timestamp, patternPartitionSizeMin);
+        String partitions = partitionItemMap.toString().substring(1, partitionItemMap.toString().length() - 1);
+
+        Map<String, String> configMap = dc.getConfig();
+        configMap.put("partitions", partitions);
+        try {
+            dc.setConfig(configMap);
+        } catch (JsonProcessingException e) {
+            e.printStackTrace();
+        }
+    }
+
+    public long setCurrentTimestamp(long currentSystemTimestamp) {
+        long currentTimstamp;
+        if (eachJoblastTimestamp != null && !eachJoblastTimestamp.equals("")) {
+            currentTimstamp = Long.parseLong(eachJoblastTimestamp) + Integer.parseInt(periodTime) * 1000;
+        } else {
+            if (dataStartTimestamp != null && !dataStartTimestamp.equals("")) {
+                currentTimstamp = Long.parseLong(dataStartTimestamp);
+            } else {
+                currentTimstamp = currentSystemTimestamp;
+            }
+        }
+        return currentTimstamp;
+    }
+
+    public void setSparkJobDO() {
+        sparkJobDO.setFile(props.getProperty("sparkJob.file"));
+        sparkJobDO.setClassName(props.getProperty("sparkJob.className"));
+
+        List<String> args = new ArrayList<>();
+        args.add(props.getProperty("sparkJob.args_1"));
+        ObjectWriter ow = new ObjectMapper().writer().withDefaultPrettyPrinter();
+        String measureJson = "";
+        try {
+            measureJson = ow.writeValueAsString(measure);
+        } catch (JsonProcessingException e) {
+            e.printStackTrace();
+        }
+        args.add(measureJson);  //partition
+        args.add(props.getProperty("sparkJob.args_3"));
+        sparkJobDO.setArgs(args);
+
+        sparkJobDO.setName(props.getProperty("sparkJob.name"));
+        sparkJobDO.setQueue(props.getProperty("sparkJob.queue"));
+        sparkJobDO.setNumExecutors(Long.parseLong(props.getProperty("sparkJob.numExecutors")));
+        sparkJobDO.setExecutorCores(Long.parseLong(props.getProperty("sparkJob.executorCores")));
+        sparkJobDO.setDriverMemory(props.getProperty("sparkJob.driverMemory"));
+        sparkJobDO.setExecutorMemory(props.getProperty("sparkJob.executorMemory"));
+
+        Conf conf = new Conf();
+        conf.setSpark_jars_packages(props.getProperty("sparkJob.spark.jars.packages"));
+        sparkJobDO.setConf(conf);
+
+        List<String> jars = new ArrayList<>();
+        jars.add(props.getProperty("sparkJob.jars_1"));
+        jars.add(props.getProperty("sparkJob.jars_2"));
+        jars.add(props.getProperty("sparkJob.jars_3"));
+        sparkJobDO.setJars(jars);
+
+        List<String> files = new ArrayList<>();
+        files.add(props.getProperty("sparkJob.files_1"));
+        sparkJobDO.setFiles(files);
+    }
+
+    public Properties getsparkJobProperties() throws IOException {
+        InputStream inputStream = null;
+        Properties prop = new Properties();
+        ;
+        try {
+            String propFileName = "sparkJob.properties";
+
+            inputStream = getClass().getClassLoader().getResourceAsStream(propFileName);
+
+            if (inputStream != null) {
+                prop.load(inputStream);
+            } else {
+                throw new FileNotFoundException("property file '" + propFileName + "' not found in the classpath");
+            }
+        } catch (Exception e) {
+            System.out.println("Exception: " + e);
+        } finally {
+            inputStream.close();
+        }
+        return prop;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-griffin/blob/f629d0f4/service/src/main/java/org/apache/griffin/core/service/GriffinController.java
----------------------------------------------------------------------
diff --git a/service/src/main/java/org/apache/griffin/core/service/GriffinController.java b/service/src/main/java/org/apache/griffin/core/service/GriffinController.java
new file mode 100644
index 0000000..7a5b5ad
--- /dev/null
+++ b/service/src/main/java/org/apache/griffin/core/service/GriffinController.java
@@ -0,0 +1,17 @@
+package org.apache.griffin.core.service;
+
+
+import org.springframework.web.bind.annotation.RequestMapping;
+import org.springframework.web.bind.annotation.RestController;
+
+
+@RestController
+public class GriffinController {
+
+    @RequestMapping("/version")
+    public String greeting() {
+        return "0.1.0";
+    }
+
+}
+

http://git-wip-us.apache.org/repos/asf/incubator-griffin/blob/f629d0f4/service/src/main/java/org/apache/griffin/core/util/AvroUtil.java
----------------------------------------------------------------------
diff --git a/service/src/main/java/org/apache/griffin/core/util/AvroUtil.java b/service/src/main/java/org/apache/griffin/core/util/AvroUtil.java
new file mode 100644
index 0000000..fb944c5
--- /dev/null
+++ b/service/src/main/java/org/apache/griffin/core/util/AvroUtil.java
@@ -0,0 +1,11 @@
+package org.apache.griffin.core.util;
+
+import org.apache.avro.Schema;
+
+public class AvroUtil {
+
+    public static Schema schemaOf(String schema) {
+        return new Schema.Parser().parse(schema);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-griffin/blob/f629d0f4/service/src/main/resources/Init_quartz.sql
----------------------------------------------------------------------
diff --git a/service/src/main/resources/Init_quartz.sql b/service/src/main/resources/Init_quartz.sql
new file mode 100644
index 0000000..6f3ee48
--- /dev/null
+++ b/service/src/main/resources/Init_quartz.sql
@@ -0,0 +1,179 @@
+#
+# In your Quartz properties file, you'll need to set
+# org.quartz.jobStore.driverDelegateClass = org.quartz.impl.jdbcjobstore.StdJDBCDelegate
+#
+#
+# By: Ron Cordell - roncordell
+#  I didn't see this anywhere, so I thought I'd post it here. This is the script from Quartz to create the tables in a MySQL database, modified to use INNODB instead of MYISAM.
+
+DROP TABLE IF EXISTS QRTZ_FIRED_TRIGGERS;
+DROP TABLE IF EXISTS QRTZ_PAUSED_TRIGGER_GRPS;
+DROP TABLE IF EXISTS QRTZ_SCHEDULER_STATE;
+DROP TABLE IF EXISTS QRTZ_LOCKS;
+DROP TABLE IF EXISTS QRTZ_SIMPLE_TRIGGERS;
+DROP TABLE IF EXISTS QRTZ_SIMPROP_TRIGGERS;
+DROP TABLE IF EXISTS QRTZ_CRON_TRIGGERS;
+DROP TABLE IF EXISTS QRTZ_BLOB_TRIGGERS;
+DROP TABLE IF EXISTS QRTZ_TRIGGERS;
+DROP TABLE IF EXISTS QRTZ_JOB_DETAILS;
+DROP TABLE IF EXISTS QRTZ_CALENDARS;
+
+CREATE TABLE QRTZ_JOB_DETAILS(
+  SCHED_NAME VARCHAR(120) NOT NULL,
+  JOB_NAME VARCHAR(200) NOT NULL,
+  JOB_GROUP VARCHAR(200) NOT NULL,
+  DESCRIPTION VARCHAR(250) NULL,
+  JOB_CLASS_NAME VARCHAR(250) NOT NULL,
+  IS_DURABLE VARCHAR(1) NOT NULL,
+  IS_NONCONCURRENT VARCHAR(1) NOT NULL,
+  IS_UPDATE_DATA VARCHAR(1) NOT NULL,
+  REQUESTS_RECOVERY VARCHAR(1) NOT NULL,
+  JOB_DATA BLOB NULL,
+  PRIMARY KEY (SCHED_NAME,JOB_NAME,JOB_GROUP))
+  ENGINE=InnoDB;
+
+CREATE TABLE QRTZ_TRIGGERS (
+  SCHED_NAME VARCHAR(120) NOT NULL,
+  TRIGGER_NAME VARCHAR(200) NOT NULL,
+  TRIGGER_GROUP VARCHAR(200) NOT NULL,
+  JOB_NAME VARCHAR(200) NOT NULL,
+  JOB_GROUP VARCHAR(200) NOT NULL,
+  DESCRIPTION VARCHAR(250) NULL,
+  NEXT_FIRE_TIME BIGINT(13) NULL,
+  PREV_FIRE_TIME BIGINT(13) NULL,
+  PRIORITY INTEGER NULL,
+  TRIGGER_STATE VARCHAR(16) NOT NULL,
+  TRIGGER_TYPE VARCHAR(8) NOT NULL,
+  START_TIME BIGINT(13) NOT NULL,
+  END_TIME BIGINT(13) NULL,
+  CALENDAR_NAME VARCHAR(200) NULL,
+  MISFIRE_INSTR SMALLINT(2) NULL,
+  JOB_DATA BLOB NULL,
+  PRIMARY KEY (SCHED_NAME,TRIGGER_NAME,TRIGGER_GROUP),
+  FOREIGN KEY (SCHED_NAME,JOB_NAME,JOB_GROUP)
+  REFERENCES QRTZ_JOB_DETAILS(SCHED_NAME,JOB_NAME,JOB_GROUP))
+  ENGINE=InnoDB;
+
+CREATE TABLE QRTZ_SIMPLE_TRIGGERS (
+  SCHED_NAME VARCHAR(120) NOT NULL,
+  TRIGGER_NAME VARCHAR(200) NOT NULL,
+  TRIGGER_GROUP VARCHAR(200) NOT NULL,
+  REPEAT_COUNT BIGINT(7) NOT NULL,
+  REPEAT_INTERVAL BIGINT(12) NOT NULL,
+  TIMES_TRIGGERED BIGINT(10) NOT NULL,
+  PRIMARY KEY (SCHED_NAME,TRIGGER_NAME,TRIGGER_GROUP),
+  FOREIGN KEY (SCHED_NAME,TRIGGER_NAME,TRIGGER_GROUP)
+  REFERENCES QRTZ_TRIGGERS(SCHED_NAME,TRIGGER_NAME,TRIGGER_GROUP))
+  ENGINE=InnoDB;
+
+CREATE TABLE QRTZ_CRON_TRIGGERS (
+  SCHED_NAME VARCHAR(120) NOT NULL,
+  TRIGGER_NAME VARCHAR(200) NOT NULL,
+  TRIGGER_GROUP VARCHAR(200) NOT NULL,
+  CRON_EXPRESSION VARCHAR(120) NOT NULL,
+  TIME_ZONE_ID VARCHAR(80),
+  PRIMARY KEY (SCHED_NAME,TRIGGER_NAME,TRIGGER_GROUP),
+  FOREIGN KEY (SCHED_NAME,TRIGGER_NAME,TRIGGER_GROUP)
+  REFERENCES QRTZ_TRIGGERS(SCHED_NAME,TRIGGER_NAME,TRIGGER_GROUP))
+  ENGINE=InnoDB;
+
+CREATE TABLE QRTZ_SIMPROP_TRIGGERS
+(
+  SCHED_NAME VARCHAR(120) NOT NULL,
+  TRIGGER_NAME VARCHAR(200) NOT NULL,
+  TRIGGER_GROUP VARCHAR(200) NOT NULL,
+  STR_PROP_1 VARCHAR(512) NULL,
+  STR_PROP_2 VARCHAR(512) NULL,
+  STR_PROP_3 VARCHAR(512) NULL,
+  INT_PROP_1 INT NULL,
+  INT_PROP_2 INT NULL,
+  LONG_PROP_1 BIGINT NULL,
+  LONG_PROP_2 BIGINT NULL,
+  DEC_PROP_1 NUMERIC(13,4) NULL,
+  DEC_PROP_2 NUMERIC(13,4) NULL,
+  BOOL_PROP_1 VARCHAR(1) NULL,
+  BOOL_PROP_2 VARCHAR(1) NULL,
+  PRIMARY KEY (SCHED_NAME,TRIGGER_NAME,TRIGGER_GROUP),
+  FOREIGN KEY (SCHED_NAME,TRIGGER_NAME,TRIGGER_GROUP)
+  REFERENCES QRTZ_TRIGGERS(SCHED_NAME,TRIGGER_NAME,TRIGGER_GROUP))
+  ENGINE=InnoDB;
+
+CREATE TABLE QRTZ_BLOB_TRIGGERS (
+  SCHED_NAME VARCHAR(120) NOT NULL,
+  TRIGGER_NAME VARCHAR(200) NOT NULL,
+  TRIGGER_GROUP VARCHAR(200) NOT NULL,
+  BLOB_DATA BLOB NULL,
+  PRIMARY KEY (SCHED_NAME,TRIGGER_NAME,TRIGGER_GROUP),
+  INDEX (SCHED_NAME,TRIGGER_NAME, TRIGGER_GROUP),
+  FOREIGN KEY (SCHED_NAME,TRIGGER_NAME,TRIGGER_GROUP)
+  REFERENCES QRTZ_TRIGGERS(SCHED_NAME,TRIGGER_NAME,TRIGGER_GROUP))
+  ENGINE=InnoDB;
+
+CREATE TABLE QRTZ_CALENDARS (
+  SCHED_NAME VARCHAR(120) NOT NULL,
+  CALENDAR_NAME VARCHAR(200) NOT NULL,
+  CALENDAR BLOB NOT NULL,
+  PRIMARY KEY (SCHED_NAME,CALENDAR_NAME))
+  ENGINE=InnoDB;
+
+CREATE TABLE QRTZ_PAUSED_TRIGGER_GRPS (
+  SCHED_NAME VARCHAR(120) NOT NULL,
+  TRIGGER_GROUP VARCHAR(200) NOT NULL,
+  PRIMARY KEY (SCHED_NAME,TRIGGER_GROUP))
+  ENGINE=InnoDB;
+
+CREATE TABLE QRTZ_FIRED_TRIGGERS (
+  SCHED_NAME VARCHAR(120) NOT NULL,
+  ENTRY_ID VARCHAR(95) NOT NULL,
+  TRIGGER_NAME VARCHAR(200) NOT NULL,
+  TRIGGER_GROUP VARCHAR(200) NOT NULL,
+  INSTANCE_NAME VARCHAR(200) NOT NULL,
+  FIRED_TIME BIGINT(13) NOT NULL,
+  SCHED_TIME BIGINT(13) NOT NULL,
+  PRIORITY INTEGER NOT NULL,
+  STATE VARCHAR(16) NOT NULL,
+  JOB_NAME VARCHAR(200) NULL,
+  JOB_GROUP VARCHAR(200) NULL,
+  IS_NONCONCURRENT VARCHAR(1) NULL,
+  REQUESTS_RECOVERY VARCHAR(1) NULL,
+  PRIMARY KEY (SCHED_NAME,ENTRY_ID))
+  ENGINE=InnoDB;
+
+CREATE TABLE QRTZ_SCHEDULER_STATE (
+  SCHED_NAME VARCHAR(120) NOT NULL,
+  INSTANCE_NAME VARCHAR(200) NOT NULL,
+  LAST_CHECKIN_TIME BIGINT(13) NOT NULL,
+  CHECKIN_INTERVAL BIGINT(13) NOT NULL,
+  PRIMARY KEY (SCHED_NAME,INSTANCE_NAME))
+  ENGINE=InnoDB;
+
+CREATE TABLE QRTZ_LOCKS (
+  SCHED_NAME VARCHAR(120) NOT NULL,
+  LOCK_NAME VARCHAR(40) NOT NULL,
+  PRIMARY KEY (SCHED_NAME,LOCK_NAME))
+  ENGINE=InnoDB;
+
+CREATE INDEX IDX_QRTZ_J_REQ_RECOVERY ON QRTZ_JOB_DETAILS(SCHED_NAME,REQUESTS_RECOVERY);
+CREATE INDEX IDX_QRTZ_J_GRP ON QRTZ_JOB_DETAILS(SCHED_NAME,JOB_GROUP);
+
+CREATE INDEX IDX_QRTZ_T_J ON QRTZ_TRIGGERS(SCHED_NAME,JOB_NAME,JOB_GROUP);
+CREATE INDEX IDX_QRTZ_T_JG ON QRTZ_TRIGGERS(SCHED_NAME,JOB_GROUP);
+CREATE INDEX IDX_QRTZ_T_C ON QRTZ_TRIGGERS(SCHED_NAME,CALENDAR_NAME);
+CREATE INDEX IDX_QRTZ_T_G ON QRTZ_TRIGGERS(SCHED_NAME,TRIGGER_GROUP);
+CREATE INDEX IDX_QRTZ_T_STATE ON QRTZ_TRIGGERS(SCHED_NAME,TRIGGER_STATE);
+CREATE INDEX IDX_QRTZ_T_N_STATE ON QRTZ_TRIGGERS(SCHED_NAME,TRIGGER_NAME,TRIGGER_GROUP,TRIGGER_STATE);
+CREATE INDEX IDX_QRTZ_T_N_G_STATE ON QRTZ_TRIGGERS(SCHED_NAME,TRIGGER_GROUP,TRIGGER_STATE);
+CREATE INDEX IDX_QRTZ_T_NEXT_FIRE_TIME ON QRTZ_TRIGGERS(SCHED_NAME,NEXT_FIRE_TIME);
+CREATE INDEX IDX_QRTZ_T_NFT_ST ON QRTZ_TRIGGERS(SCHED_NAME,TRIGGER_STATE,NEXT_FIRE_TIME);
+CREATE INDEX IDX_QRTZ_T_NFT_MISFIRE ON QRTZ_TRIGGERS(SCHED_NAME,MISFIRE_INSTR,NEXT_FIRE_TIME);
+CREATE INDEX IDX_QRTZ_T_NFT_ST_MISFIRE ON QRTZ_TRIGGERS(SCHED_NAME,MISFIRE_INSTR,NEXT_FIRE_TIME,TRIGGER_STATE);
+CREATE INDEX IDX_QRTZ_T_NFT_ST_MISFIRE_GRP ON QRTZ_TRIGGERS(SCHED_NAME,MISFIRE_INSTR,NEXT_FIRE_TIME,TRIGGER_GROUP,TRIGGER_STATE);
+
+CREATE INDEX IDX_QRTZ_FT_TRIG_INST_NAME ON QRTZ_FIRED_TRIGGERS(SCHED_NAME,INSTANCE_NAME);
+CREATE INDEX IDX_QRTZ_FT_INST_JOB_REQ_RCVRY ON QRTZ_FIRED_TRIGGERS(SCHED_NAME,INSTANCE_NAME,REQUESTS_RECOVERY);
+CREATE INDEX IDX_QRTZ_FT_J_G ON QRTZ_FIRED_TRIGGERS(SCHED_NAME,JOB_NAME,JOB_GROUP);
+CREATE INDEX IDX_QRTZ_FT_JG ON QRTZ_FIRED_TRIGGERS(SCHED_NAME,JOB_GROUP);
+CREATE INDEX IDX_QRTZ_FT_T_G ON QRTZ_FIRED_TRIGGERS(SCHED_NAME,TRIGGER_NAME,TRIGGER_GROUP);
+CREATE INDEX IDX_QRTZ_FT_TG ON QRTZ_FIRED_TRIGGERS(SCHED_NAME,TRIGGER_GROUP);
+
+commit;

http://git-wip-us.apache.org/repos/asf/incubator-griffin/blob/f629d0f4/service/src/main/resources/application-prod.properties
----------------------------------------------------------------------
diff --git a/service/src/main/resources/application-prod.properties b/service/src/main/resources/application-prod.properties
new file mode 100644
index 0000000..9e91c74
--- /dev/null
+++ b/service/src/main/resources/application-prod.properties
@@ -0,0 +1,19 @@
+spring.datasource.url= jdbc:mysql://localhost:3306/griffin
+spring.datasource.username =root
+spring.datasource.password =1qaz
+#spring.datasource.driver-class-name=com.mysql.jdbc.Driver
+
+spring.jpa.hibernate.ddl-auto = update
+spring.jpa.show-sql=true
+spring.jpa.properties.hibernate.dialect=org.hibernate.dialect.MySQL5Dialect
+
+
+# Naming strategy
+spring.jpa.hibernate.naming-strategy = org.hibernate.cfg.ImprovedNamingStrategy
+
+# hive metastore
+hive.metastore.uris = thrift://10.9.246.187:9083
+hive.metastore.dbname = default
+
+# kafka schema registry
+kafka.schema.registry.url = http://10.65.159.119:8081
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-griffin/blob/f629d0f4/service/src/main/resources/application.properties
----------------------------------------------------------------------
diff --git a/service/src/main/resources/application.properties b/service/src/main/resources/application.properties
new file mode 100644
index 0000000..b836678
--- /dev/null
+++ b/service/src/main/resources/application.properties
@@ -0,0 +1,22 @@
+spring.datasource.url= jdbc:mysql://localhost:3306/metastore
+spring.datasource.username =root
+spring.datasource.password =cxr123
+
+spring.datasource.driver-class-name=com.mysql.jdbc.Driver
+
+## Hibernate ddl auto (validate,create, create-drop, update)
+
+spring.jpa.hibernate.ddl-auto = create-drop
+spring.jpa.show-sql=true
+spring.jpa.properties.hibernate.dialect=org.hibernate.dialect.MySQL5Dialect
+#
+#
+## Naming strategy
+spring.jpa.hibernate.naming-strategy = org.hibernate.cfg.ImprovedNamingStrategy
+
+# hive metastore
+hive.metastore.uris = thrift://10.9.246.187:9083
+hive.metastore.dbname = default
+
+# kafka schema registry
+kafka.schema.registry.url = http://10.65.159.119:8081
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-griffin/blob/f629d0f4/service/src/main/resources/public/apidocs/bark.json
----------------------------------------------------------------------
diff --git a/service/src/main/resources/public/apidocs/bark.json b/service/src/main/resources/public/apidocs/bark.json
new file mode 100644
index 0000000..9a01d6e
--- /dev/null
+++ b/service/src/main/resources/public/apidocs/bark.json
@@ -0,0 +1,737 @@
+{
+    "swagger": "2.0",
+    "info": {
+        "title": "Griffin API",
+        "description": "Move your app forward with the Griffin API",
+        "version": "1.0.0"
+    },
+    "host": "localhost:8080",
+    "schemes": [
+        "http"
+    ],
+    "produces": [
+        "application/json"
+    ],
+    "paths": {
+        "/api/v1/dq/metrics/heatmap": {
+            "get": {
+                "summary": "User Profile",
+                "description": "heatmap",
+                "tags": [
+                    "health"
+                ],
+                "responses": {
+                    "200": {
+                        "description": "success"
+                    },
+                    "default": {
+                        "description": "Unexpected error"
+                    }
+                }
+            }
+        },
+        "/js/mock_data/statistics.json": {
+            "get": {
+                "summary": "User Profile",
+                "description": "statistics",
+                "tags": [
+                    "health"
+                ],
+                "responses": {
+                    "200": {
+                        "description": "success"
+                    },
+                    "default": {
+                        "description": "Unexpected error"
+                    }
+                }
+            }
+        },
+        "/js/mock_data/rulemetric.json": {
+            "get": {
+                "summary": "User Profile",
+                "description": "rulemetric",
+                "responses": {
+                    "200": {
+                        "description": "success"
+                    },
+                    "default": {
+                        "description": "Unexpected error"
+                    }
+                }
+            }
+        },
+        "/api/v1/dq/metrics/briefmetrics": {
+            "get": {
+                "summary": "User Profile",
+                "description": "Metrics on side bar",
+                "tags": [
+                    "health"
+                ],
+                "responses": {
+                    "200": {
+                        "description": "success"
+                    },
+                    "default": {
+                        "description": "Unexpected error"
+                    }
+                }
+            }
+        },
+        "/api/v1/dataassets/srctree": {
+            "get": {
+                "summary": "User Profile",
+                "description": "Metrics on side bar",
+                "tags": [
+                    "createrule-ac"
+                ],
+                "responses": {
+                    "200": {
+                        "description": "success"
+                    },
+                    "default": {
+                        "description": "Unexpected error"
+                    }
+                }
+            }
+        },
+        "/api/v1/dataassets/{id}": {
+            "get": {
+                "summary": "User Profile",
+                "description": "Get specfied data in datasets",
+                "parameters": [
+                    {
+                        "name": "id",
+                        "in": "path",
+                        "required": true,
+                        "description": "ID like 1,2,3...",
+                        "type": "integer"
+                    }
+                ],
+                "tags": [
+                    "createrule-ac"
+                ],
+                "responses": {
+                    "200": {
+                        "description": "success"
+                    },
+                    "default": {
+                        "description": "Unexpected error"
+                    }
+                }
+            }
+        },
+        "/api/v1/dq/metrics/complete/{id}": {
+            "get": {
+                "summary": "User Profile",
+                "description": "metrics/complete",
+                "parameters": [
+                    {
+                        "name": "id",
+                        "in": "path",
+                        "required": true,
+                        "description": "ID like accuracy_viewitem_queue, accuracy_bid_new_queue,...",
+                        "type": "string"
+                    }
+                ],
+                "tags": [
+                    "metrics/Bullseye"
+                ],
+                "responses": {
+                    "200": {
+                        "description": "success"
+                    },
+                    "default": {
+                        "description": "Unexpected error"
+                    }
+                }
+            }
+        },
+        "/api/v1/dq/metrics/dashboard/Bullseye": {
+            "get": {
+                "summary": "User Profile",
+                "description": "dashboard",
+                "tags": [
+                    "metrics/Bullseye"
+                ],
+                "responses": {
+                    "200": {
+                        "description": "success"
+                    },
+                    "default": {
+                        "description": "Unexpected error"
+                    }
+                }
+            }
+        },
+        "/api/v1/dq/metrics/dashboard": {
+            "get": {
+                "summary": "User Profile",
+                "description": "dashboard",
+                "tags": [
+                    "metrics"
+                ],
+                "responses": {
+                    "200": {
+                        "description": "success"
+                    },
+                    "default": {
+                        "description": "Unexpected error"
+                    }
+                }
+            }
+        },
+        "/api/v1/model/allModels": {
+            "get": {
+                "summary": "User Profile",
+                "description": "allModels",
+                "tags": [
+                    "rules"
+                ],
+                "responses": {
+                    "200": {
+                        "description": "success"
+                    },
+                    "default": {
+                        "description": "Unexpected error"
+                    }
+                }
+            }
+        },
+        "/api/v1/model/getAnomalyModel/{id}": {
+            "get": {
+                "summary": "User Profile",
+                "description": "GetAnomalyModel by id",
+                "parameters": [
+                    {
+                        "name": "id",
+                        "in": "path",
+                        "required": true,
+                        "description": "ID like vad1, ...",
+                        "type": "string"
+                    }
+                ],
+                "tags": [
+                    "viewrule"
+                ],
+                "responses": {
+                    "200": {
+                        "description": "success"
+                    },
+                    "default": {
+                        "description": "Unexpected error"
+                    }
+                }
+            }
+        },
+        "/api/v1/model/getPublishModel/{id}": {
+            "get": {
+                "summary": "User Profile",
+                "description": "GetPublishModel by id",
+                "parameters": [
+                    {
+                        "name": "id",
+                        "in": "path",
+                        "required": true,
+                        "description": "ID like vad1, ...",
+                        "type": "string"
+                    }
+                ],
+                "tags": [
+                    "viewrule"
+                ],
+                "responses": {
+                    "200": {
+                        "description": "success"
+                    },
+                    "default": {
+                        "description": "Unexpected error"
+                    }
+                }
+            }
+        },
+        "/api/v1/model/getAccuracyModel/{id}": {
+            "get": {
+                "summary": "User Profile",
+                "description": "GetAccuracyModel by id",
+                "parameters": [
+                    {
+                        "name": "id",
+                        "in": "path",
+                        "required": true,
+                        "description": "ID like test_accuracy, ...",
+                        "type": "string"
+                    }
+                ],
+                "tags": [
+                    "viewrule"
+                ],
+                "responses": {
+                    "200": {
+                        "description": "success"
+                    },
+                    "default": {
+                        "description": "Unexpected error"
+                    }
+                }
+            }
+        },
+        "/api/v1/model/getValidityModel/{id}": {
+            "get": {
+                "summary": "User Profile",
+                "description": "GetValidityModel by id",
+                "parameters": [
+                    {
+                        "name": "id",
+                        "in": "path",
+                        "required": true,
+                        "description": "ID like vad1, ...",
+                        "type": "string"
+                    }
+                ],
+                "tags": [
+                    "viewrule"
+                ],
+                "responses": {
+                    "200": {
+                        "description": "success"
+                    },
+                    "default": {
+                        "description": "Unexpected error"
+                    }
+                }
+            }
+        },
+        "/api/v1/model/newAccuracyModel": {
+            "post": {
+                "summary": "User Profile",
+                "description": "newAccuracyModel",
+                "parameters": [
+                    {
+                        "name": "body",
+                        "in": "body",
+                        "required": true,
+                        "description": "Please only change the name! ",
+                        "schema": {
+                            "$ref": "#/definitions/newAccuracyModel"
+                        }
+                    }
+                ],
+                "tags": [
+                    "viewrule"
+                ],
+                "responses": {
+                    "200": {
+                        "description": "Profile information for a user"
+                    },
+                    "default": {
+                        "description": "Unexpected error"
+                    }
+                }
+            }
+        },
+        "/api/v1/model/newValidityModel": {
+            "post": {
+                "summary": "User Profile",
+                "description": "newValidityModel",
+                "parameters": [
+                    {
+                        "name": "body",
+                        "in": "body",
+                        "required": true,
+                        "description": "Please only change the name! ",
+                        "schema": {
+                            "$ref": "#/definitions/newValidityModel"
+                        }
+                    }
+                ],
+                "tags": [
+                    "viewrule"
+                ],
+                "responses": {
+                    "200": {
+                        "description": "Profile information for a user"
+                    },
+                    "default": {
+                        "description": "Unexpected error"
+                    }
+                }
+            }
+        },
+        "/api/v1/model/newAnomalyModel": {
+            "post": {
+                "summary": "User Profile",
+                "description": "newAnomalyModel",
+                "parameters": [
+                    {
+                        "name": "body",
+                        "in": "body",
+                        "required": true,
+                        "description": "Please only change the name! ",
+                        "schema": {
+                            "$ref": "#/definitions/newAnomalyModel"
+                        }
+                    }
+                ],
+                "tags": [
+                    "viewrule"
+                ],
+                "responses": {
+                    "200": {
+                        "description": "Profile information for a user"
+                    },
+                    "default": {
+                        "description": "Unexpected error"
+                    }
+                }
+            }
+        },
+        "/api/v1/model/newPublishModel": {
+            "post": {
+                "summary": "User Profile",
+                "description": "newPublishModel",
+                "parameters": [
+                    {
+                        "name": "body",
+                        "in": "body",
+                        "required": true,
+                        "description": "Please only change the name! ",
+                        "schema": {
+                            "$ref": "#/definitions/newPublishModel"
+                        }
+                    }
+                ],
+                "tags": [
+                    "viewrule"
+                ],
+                "responses": {
+                    "200": {
+                        "description": "Profile information for a user"
+                    },
+                    "default": {
+                        "description": "Unexpected error"
+                    }
+                }
+            }
+        },
+        "/api/v1/model/deleteModel/{id}": {
+            "delete": {
+                "summary": "User Profile",
+                "description": "Get specfied data in datasets",
+                "parameters": [
+                    {
+                        "name": "id",
+                        "in": "path",
+                        "required": true,
+                        "description": "ID like wx_publish, ...",
+                        "type": "string"
+                    }
+                ],
+                "tags": [
+                    "viewrule"
+                ],
+                "responses": {
+                    "200": {
+                        "description": "success"
+                    },
+                    "default": {
+                        "description": "Unexpected error"
+                    }
+                }
+            }
+        }
+    },
+    "definitions": {
+        "mappingsItem":{
+            "type": "object",
+            "properties": {
+                "target": {
+                    "type": "string",
+                    "enum": ["sitespeed.key"]
+                },
+                "src": {
+                    "type": "string",
+                    "enum": ["dw_bid.uid"]
+                },
+                "matchMethod": {
+                    "type": "string",
+                    "enum": ["EXACT"]
+                },
+                "isPk": {
+                    "type": "boolean",
+                    "enum": [true]
+                }
+            }
+        },
+        "newAccuracyModel": {
+            "type": "object",
+            "properties": {
+                "basic": {
+                    "type": "object",
+                    "properties": {
+                        "type": {
+                            "type": "string",
+                            "enum": ["0"]
+                        },
+                        "system": {
+                            "type": "string",
+                            "enum": ["2"]
+                        },
+                        "threshold": {
+                            "type": "integer",
+                            "enum": [90]
+                        },
+                        "scheduleType": {
+                            "type": "string",
+                            "enum": ["0"]
+                        },
+                        "owner": {
+                            "type": "string",
+                            "enum": ["xwang21"]
+                        },
+                        "name": {
+                            "type": "string",
+                            "enum": ["wx_ac"]
+                        },
+                        "desc": {
+                            "type": "string",
+                            "enum": ["ggggg"]
+                        },
+                        "email": {
+                            "type": "string",
+                            "enum": ["g@g"]
+                        },
+                        "dataaset": {
+                            "type": "string",
+                            "enum": ["sitespeed"]
+                        },
+                        "dataasetId": {
+                            "type": "integer",
+                            "enum": [21]
+                        }
+                    }
+                },
+                "extra": {
+                    "type": "object",
+                    "properties": {
+                        "srcDb": {
+                            "type": "string",
+                            "enum": ["Apollo"]
+                        },
+                        "srcDataSet": {
+                            "type": "string",
+                            "enum": ["Bullseye"]
+                        },
+                        "targetDb": {
+                            "type": "string",
+                            "enum": ["Apollo"]
+                        },
+                        "targetDataSet": {
+                            "type": "string",
+                            "enum": ["SiteSpeed"]
+                        }
+                    }
+                },
+                "mappings": {
+                    "type": "array",
+                    "items":{
+                        "$ref": "mappingsItem"
+                    }
+                }
+            }
+        },
+        "newValidityModel": {
+            "type": "object",
+            "properties": {
+                "basic": {
+                    "type": "object",
+                    "properties": {
+                        "type": {
+                            "type": "string",
+                            "enum": ["1"]
+                        },
+                        "system": {
+                            "type": "string",
+                            "enum": ["3"]
+                        },
+                        "scheduleType": {
+                            "type": "string",
+                            "enum": ["1"]
+                        },
+                        "owner": {
+                            "type": "string",
+                            "enum": ["xwang21"]
+                        },
+                        "name": {
+                            "type": "string",
+                            "enum": ["wx_valid"]
+                        },
+                        "desc": {
+                            "type": "string",
+                            "enum": ["wwwwww"]
+                        },
+                        "threshold": {
+                            "type": "integer",
+                            "enum": [60]
+                        },
+                        "email": {
+                            "type": "string",
+                            "enum": ["w@f"]
+                        },
+                        "dataaset": {
+                            "type": "string",
+                            "enum": ["ubi_event"]
+                        },
+                        "dataasetId": {
+                            "type": "integer",
+                            "enum": [22]
+                        }
+                    }
+                },
+                "extra": {
+                    "type": "object",
+                    "properties": {
+                        "srcDb": {
+                            "type": "string",
+                            "enum": ["Apollo"]
+                        },
+                        "srcDataSet": {
+                            "type": "string",
+                            "enum": ["Sojourner"]
+                        }
+                    }
+                },
+                "vaType": {
+                    "type": "string",
+                    "enum": ["5"]
+                },
+                "column": {
+                    "type": "string",
+                    "enum": ["guid"]
+                }
+            }
+        },
+        "newAnomalyModel": {
+            "type": "object",
+            "properties": {
+                "basic": {
+                    "type": "object",
+                    "properties": {
+                        "type": {
+                            "type": "string",
+                            "enum": ["2"]
+                        },
+                        "system": {
+                            "type": "string",
+                            "enum": ["4"]
+                        },
+                        "scheduleType": {
+                            "type": "string",
+                            "enum": ["0"]
+                        },
+                        "owner": {
+                            "type": "string",
+                            "enum": ["xwang21"]
+                        },
+                        "name": {
+                            "type": "string",
+                            "enum": ["wx_detec"]
+                        },
+                        "desc": {
+                            "type": "string",
+                            "enum": ["wwwwww"]
+                        },
+                        "threshold": {
+                            "type": "integer",
+                            "enum": [30]
+                        },
+                        "email": {
+                            "type": "string",
+                            "enum": ["w@f"]
+                        },
+                        "dataaset": {
+                            "type": "string",
+                            "enum": ["ubi_event"]
+                        },
+                        "dataasetId": {
+                            "type": "integer",
+                            "enum": [22]
+                        }
+                    }
+                },
+                "extra": {
+                    "type": "object",
+                    "properties": {
+                        "srcDb": {
+                            "type": "string",
+                            "enum": ["Apollo"]
+                        },
+                        "srcDataSet": {
+                            "type": "string",
+                            "enum": ["Sojourner"]
+                        }
+                    }
+                },
+                "anType": {
+                    "type": "string",
+                    "enum": ["3"]
+                }
+            }
+        },
+        "newPublishModel": {
+            "type": "object",
+            "properties": {
+                "basic": {
+                    "type": "object",
+                    "properties": {
+                        "type": {
+                            "type": "string",
+                            "enum": ["3"]
+                        },
+                        "system": {
+                            "type": "string",
+                            "enum": ["3"]
+                        },
+                        "scheduleType": {
+                            "type": "string",
+                            "enum": ["0"]
+                        },
+                        "owner": {
+                            "type": "string",
+                            "enum": ["xwang21"]
+                        },
+                        "name": {
+                            "type": "string",
+                            "enum": ["wx_publish"]
+                        },
+                        "desc": {
+                            "type": "string",
+                            "enum": ["wwwwww"]
+                        },
+                        "dataaset": {
+                            "type": "string",
+                            "enum": ["sssss"]
+                        },
+                        "threshold": {
+                            "type": "integer",
+                            "enum": [80]
+                        },
+                        "email": {
+                            "type": "string",
+                            "enum": ["w@f"]
+                        }
+                    }
+                },
+                "extra": {
+                    "type": "object",
+                    "properties": {
+                        "publishUrl": {
+                            "type": "string",
+                            "enum": ["http://dq.vip.ebay.com/api/v1/publishmetric/wx_publish"]
+                        }
+                    }
+                }
+            }
+        }
+    }
+}