You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@atlas.apache.org by ma...@apache.org on 2017/01/27 00:39:58 UTC

incubator-atlas git commit: ATLAS-1312: Update QuickStart to use the v2 APIs for types and entities creation

Repository: incubator-atlas
Updated Branches:
  refs/heads/master 6ccba52c2 -> 57f4f79d6


ATLAS-1312: Update QuickStart to use the v2 APIs for types and entities creation

Signed-off-by: Madhan Neethiraj <ma...@apache.org>


Project: http://git-wip-us.apache.org/repos/asf/incubator-atlas/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-atlas/commit/57f4f79d
Tree: http://git-wip-us.apache.org/repos/asf/incubator-atlas/tree/57f4f79d
Diff: http://git-wip-us.apache.org/repos/asf/incubator-atlas/diff/57f4f79d

Branch: refs/heads/master
Commit: 57f4f79d6f8c39ee2504b23e27b1082b667e8cb1
Parents: 6ccba52
Author: Sarath Subramanian <ss...@hortonworks.com>
Authored: Mon Nov 21 10:18:55 2016 -0800
Committer: Madhan Neethiraj <ma...@apache.org>
Committed: Thu Jan 26 16:39:52 2017 -0800

----------------------------------------------------------------------
 .../java/org/apache/atlas/AtlasBaseClient.java  |   1 -
 .../org/apache/atlas/AtlasLineageClientV2.java  |  65 +++
 distro/src/bin/quick_start.py                   |   6 +-
 distro/src/bin/quick_start_v1.py                |  44 ++
 .../org/apache/atlas/type/AtlasTypeUtil.java    |  43 +-
 release-log.txt                                 |   1 +
 .../org/apache/atlas/examples/QuickStart.java   |   2 +-
 .../org/apache/atlas/examples/QuickStartV2.java | 569 +++++++++++++++++++
 .../apache/atlas/examples/QuickStartV2IT.java   | 180 ++++++
 .../atlas/web/resources/BaseResourceIT.java     |   4 +
 10 files changed, 908 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/57f4f79d/client/src/main/java/org/apache/atlas/AtlasBaseClient.java
----------------------------------------------------------------------
diff --git a/client/src/main/java/org/apache/atlas/AtlasBaseClient.java b/client/src/main/java/org/apache/atlas/AtlasBaseClient.java
index 8162900..9b69991 100644
--- a/client/src/main/java/org/apache/atlas/AtlasBaseClient.java
+++ b/client/src/main/java/org/apache/atlas/AtlasBaseClient.java
@@ -294,7 +294,6 @@ public abstract class AtlasBaseClient {
 
             if (clientResponse.getStatus() == api.getExpectedStatus().getStatusCode()) {
                 if (null == responseType) {
-                    LOG.warn("No response type specified, returning null");
                     return null;
                 }
                 try {

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/57f4f79d/client/src/main/java/org/apache/atlas/AtlasLineageClientV2.java
----------------------------------------------------------------------
diff --git a/client/src/main/java/org/apache/atlas/AtlasLineageClientV2.java b/client/src/main/java/org/apache/atlas/AtlasLineageClientV2.java
new file mode 100644
index 0000000..ac870d4
--- /dev/null
+++ b/client/src/main/java/org/apache/atlas/AtlasLineageClientV2.java
@@ -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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.atlas;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.sun.jersey.api.client.WebResource;
+import com.sun.jersey.core.util.MultivaluedMapImpl;
+import org.apache.atlas.model.lineage.AtlasLineageInfo;
+import org.apache.atlas.model.lineage.AtlasLineageInfo.LineageDirection;
+import org.apache.commons.configuration.Configuration;
+import org.apache.hadoop.security.UserGroupInformation;
+
+import javax.ws.rs.HttpMethod;
+import javax.ws.rs.core.MultivaluedMap;
+import javax.ws.rs.core.Response;
+
+public class AtlasLineageClientV2 extends AtlasBaseClient {
+
+    private static final String  LINEAGE_URI  = BASE_URI + "v2/lineage/%s/";
+    private static final APIInfo LINEAGE_INFO = new APIInfo(LINEAGE_URI, HttpMethod.GET, Response.Status.OK);
+
+    public AtlasLineageClientV2(String[] baseUrl, String[] basicAuthUserNamePassword) {
+        super(baseUrl, basicAuthUserNamePassword);
+    }
+
+    public AtlasLineageClientV2(String... baseUrls) throws AtlasException {
+        super(baseUrls);
+    }
+
+    public AtlasLineageClientV2(UserGroupInformation ugi, String doAsUser, String... baseUrls) {
+        super(ugi, doAsUser, baseUrls);
+    }
+
+    protected AtlasLineageClientV2() {
+        super();
+    }
+
+    @VisibleForTesting
+    AtlasLineageClientV2(WebResource service, Configuration configuration) {
+        super(service, configuration);
+    }
+
+    public AtlasLineageInfo getLineageInfo(final String guid, final LineageDirection direction, final int depth) throws AtlasServiceException {
+        MultivaluedMap<String, String> queryParams = new MultivaluedMapImpl();
+        queryParams.add("direction", direction.toString());
+        queryParams.add("depth", String.valueOf(depth));
+
+        return callAPI(formatPathForPathParams(LINEAGE_INFO, guid), AtlasLineageInfo.class, queryParams);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/57f4f79d/distro/src/bin/quick_start.py
----------------------------------------------------------------------
diff --git a/distro/src/bin/quick_start.py b/distro/src/bin/quick_start.py
index 14c8464..9e8b33c 100755
--- a/distro/src/bin/quick_start.py
+++ b/distro/src/bin/quick_start.py
@@ -27,16 +27,16 @@ def main():
     jvm_opts_list = cmdline.setup_jvm_opts_list(conf_dir, 'quick_start.log')
     atlas_classpath = cmdline.get_atlas_classpath(conf_dir)
 
-    process = mc.java("org.apache.atlas.examples.QuickStart", sys.argv[1:], atlas_classpath, jvm_opts_list)
+    process = mc.java("org.apache.atlas.examples.QuickStartV2", sys.argv[1:], atlas_classpath, jvm_opts_list)
     return process.wait()
 
 if __name__ == '__main__':
     try:
         returncode = main()
         if returncode == 0:
-            print "Example data added to Apache Atlas Server!!!\n"
+            print "Sample data added to Apache Atlas Server.\n"
         else:
-            print "No data was added to the Apache Atlas Server.\n"
+            print "No sample data added to Apache Atlas Server.\n"
     except Exception as e:
         print "Exception: %s " % str(e)
         returncode = -1

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/57f4f79d/distro/src/bin/quick_start_v1.py
----------------------------------------------------------------------
diff --git a/distro/src/bin/quick_start_v1.py b/distro/src/bin/quick_start_v1.py
new file mode 100755
index 0000000..e9997b1
--- /dev/null
+++ b/distro/src/bin/quick_start_v1.py
@@ -0,0 +1,44 @@
+#!/usr/bin/env python
+
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+import os
+import sys
+
+import atlas_config as mc
+import atlas_client_cmdline as cmdline
+
+def main():
+
+    conf_dir = cmdline.setup_conf_dir()
+    jvm_opts_list = cmdline.setup_jvm_opts_list(conf_dir, 'quick_start_v1.log')
+    atlas_classpath = cmdline.get_atlas_classpath(conf_dir)
+
+    process = mc.java("org.apache.atlas.examples.QuickStart", sys.argv[1:], atlas_classpath, jvm_opts_list)
+    return process.wait()
+
+if __name__ == '__main__':
+    try:
+        returncode = main()
+        if returncode == 0:
+            print "Example data added to Apache Atlas Server!!!\n"
+        else:
+            print "No data was added to the Apache Atlas Server.\n"
+    except Exception as e:
+        print "Exception: %s " % str(e)
+        returncode = -1
+
+    sys.exit(returncode)

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/57f4f79d/intg/src/main/java/org/apache/atlas/type/AtlasTypeUtil.java
----------------------------------------------------------------------
diff --git a/intg/src/main/java/org/apache/atlas/type/AtlasTypeUtil.java b/intg/src/main/java/org/apache/atlas/type/AtlasTypeUtil.java
index c866946..089bebee 100644
--- a/intg/src/main/java/org/apache/atlas/type/AtlasTypeUtil.java
+++ b/intg/src/main/java/org/apache/atlas/type/AtlasTypeUtil.java
@@ -17,6 +17,7 @@
  */
 package org.apache.atlas.type;
 
+import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.ImmutableSet;
 
 import org.apache.atlas.AtlasErrorCode;
@@ -35,6 +36,7 @@ import org.apache.commons.collections.CollectionUtils;
 import org.apache.commons.lang.StringUtils;
 
 import java.util.Collections;
+import java.util.HashMap;
 import java.util.HashSet;
 import java.util.LinkedList;
 import java.util.List;
@@ -50,6 +52,8 @@ import static org.apache.atlas.model.typedef.AtlasBaseTypeDef.ATLAS_TYPE_ARRAY_S
 import static org.apache.atlas.model.typedef.AtlasBaseTypeDef.ATLAS_TYPE_MAP_KEY_VAL_SEP;
 import static org.apache.atlas.model.typedef.AtlasBaseTypeDef.ATLAS_TYPE_MAP_PREFIX;
 import static org.apache.atlas.model.typedef.AtlasBaseTypeDef.ATLAS_TYPE_MAP_SUFFIX;
+import static org.apache.atlas.model.typedef.AtlasStructDef.AtlasConstraintDef.CONSTRAINT_PARAM_REF_ATTRIBUTE;
+import static org.apache.atlas.model.typedef.AtlasStructDef.AtlasConstraintDef.CONSTRAINT_TYPE_MAPPED_FROM_REF;
 
 /**
  * Utility methods for AtlasType/AtlasTypeDef.
@@ -170,6 +174,41 @@ public class AtlasTypeUtil {
             Collections.<AtlasStructDef.AtlasConstraintDef>emptyList());
     }
 
+    public static AtlasAttributeDef createListRequiredAttrDef(String name, String dataType) {
+        return new AtlasAttributeDef(name, dataType, false,
+                Cardinality.LIST, 1, Integer.MAX_VALUE,
+                false, true,
+                Collections.<AtlasStructDef.AtlasConstraintDef>emptyList());
+    }
+
+    public static AtlasAttributeDef createOptionalListAttrDef(String name, String dataType) {
+        return new AtlasAttributeDef(name, dataType, true,
+                Cardinality.LIST, 1, Integer.MAX_VALUE,
+                false, true,
+                Collections.<AtlasStructDef.AtlasConstraintDef>emptyList());
+    }
+
+    public static AtlasAttributeDef createRequiredListAttrDefWithConstraint(String name, String dataType, String type, Map param) {
+        AtlasAttributeDef ret = AtlasTypeUtil.createListRequiredAttrDef(name, dataType);
+        ret.addConstraint(new AtlasStructDef.AtlasConstraintDef(type, param));
+
+        return ret;
+    }
+
+    public static AtlasAttributeDef createRequiredAttrDefWithConstraint(String name, String typeName, String type, Map param) {
+        AtlasAttributeDef ret = AtlasTypeUtil.createRequiredAttrDef(name, typeName);
+        ret.addConstraint(new AtlasStructDef.AtlasConstraintDef(type, param));
+
+        return ret;
+    }
+
+    public static AtlasAttributeDef createOptionalAttrDefWithConstraint(String name, String typeName, String type, Map param) {
+        AtlasAttributeDef ret = AtlasTypeUtil.createOptionalAttrDef(name, typeName);
+        ret.addConstraint(new AtlasStructDef.AtlasConstraintDef(type, param));
+
+        return ret;
+    }
+
     public static AtlasAttributeDef createUniqueRequiredAttrDef(String name, AtlasType dataType) {
         return new AtlasAttributeDef(name, dataType.getTypeName(), false,
             Cardinality.SINGLE, 1, 1,
@@ -204,7 +243,7 @@ public class AtlasTypeUtil {
     }
 
     public static AtlasClassificationDef createTraitTypeDef(String name, String description, String version, ImmutableSet<String> superTypes, AtlasAttributeDef... attrDefs) {
-        return new AtlasClassificationDef(name, description, "1.0", Arrays.asList(attrDefs), superTypes);
+        return new AtlasClassificationDef(name, description, version, Arrays.asList(attrDefs), superTypes);
     }
 
     public static AtlasStructDef createStructTypeDef(String name, AtlasAttributeDef... attrDefs) {
@@ -227,7 +266,7 @@ public class AtlasTypeUtil {
 
     public static AtlasEntityDef createClassTypeDef(String name, String description, String version,
         ImmutableSet<String> superTypes, AtlasAttributeDef... attrDefs) {
-        return new AtlasEntityDef(name, description, "1.0", Arrays.asList(attrDefs), superTypes);
+        return new AtlasEntityDef(name, description, version, Arrays.asList(attrDefs), superTypes);
     }
 
     public static AtlasTypesDef getTypesDef(List<AtlasEnumDef> enums,

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/57f4f79d/release-log.txt
----------------------------------------------------------------------
diff --git a/release-log.txt b/release-log.txt
index 108d86b..b04dbb8 100644
--- a/release-log.txt
+++ b/release-log.txt
@@ -9,6 +9,7 @@ ATLAS-1060 Add composite indexes for exact match performance improvements for al
 ATLAS-1127 Modify creation and modification timestamps to Date instead of Long(sumasai)
 
 ALL CHANGES:
+ATLAS-1312 Update QuickStart to use the v2 APIs for types and entities creation (sarath.kum4r@gmail.com via mneethiraj)
 ATLAS-1498 added unit-tests to validate handling of array/map/struct attributes in entity create/update (sumasai via mneethiraj)
 ATLAS-1114 Performance improvements for create/update entity (jnhagelb)
 ATLAS-1403 Perf and stability improvements to DSL search and lineage query execution (sarath.kum4r@gmail.com via mneethiraj)

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/57f4f79d/webapp/src/main/java/org/apache/atlas/examples/QuickStart.java
----------------------------------------------------------------------
diff --git a/webapp/src/main/java/org/apache/atlas/examples/QuickStart.java b/webapp/src/main/java/org/apache/atlas/examples/QuickStart.java
index 8322bc6..a5d1492 100755
--- a/webapp/src/main/java/org/apache/atlas/examples/QuickStart.java
+++ b/webapp/src/main/java/org/apache/atlas/examples/QuickStart.java
@@ -108,7 +108,7 @@ public class QuickStart {
         Configuration configuration = ApplicationProperties.get();
         String[] urls = configuration.getStringArray(ATLAS_REST_ADDRESS);
         if (urls == null || urls.length == 0) {
-            System.out.println("Usage: quick_start.py <atlas endpoint of format <http/https>://<atlas-fqdn>:<atlas port> like http://localhost:21000>");
+            System.out.println("Usage: quick_start_v1.py <atlas endpoint of format <http/https>://<atlas-fqdn>:<atlas port> like http://localhost:21000>");
             System.exit(-1);
         }
 

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/57f4f79d/webapp/src/main/java/org/apache/atlas/examples/QuickStartV2.java
----------------------------------------------------------------------
diff --git a/webapp/src/main/java/org/apache/atlas/examples/QuickStartV2.java b/webapp/src/main/java/org/apache/atlas/examples/QuickStartV2.java
new file mode 100755
index 0000000..87e37e3
--- /dev/null
+++ b/webapp/src/main/java/org/apache/atlas/examples/QuickStartV2.java
@@ -0,0 +1,569 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.atlas.examples;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableSet;
+import com.sun.jersey.core.util.MultivaluedMapImpl;
+import org.apache.atlas.ApplicationProperties;
+import org.apache.atlas.AtlasClient;
+import org.apache.atlas.AtlasDiscoveryClientV2;
+import org.apache.atlas.AtlasEntitiesClientV2;
+import org.apache.atlas.AtlasException;
+import org.apache.atlas.AtlasLineageClientV2;
+import org.apache.atlas.AtlasServiceException;
+import org.apache.atlas.AtlasTypedefClientV2;
+import org.apache.atlas.model.SearchFilter;
+import org.apache.atlas.model.discovery.AtlasSearchResult;
+import org.apache.atlas.model.discovery.AtlasSearchResult.AtlasFullTextResult;
+import org.apache.atlas.model.discovery.AtlasSearchResult.AttributeSearchResult;
+import org.apache.atlas.model.instance.AtlasClassification;
+import org.apache.atlas.model.instance.AtlasEntity;
+import org.apache.atlas.model.instance.AtlasEntityHeader;
+import org.apache.atlas.model.instance.AtlasEntityHeaderWithAssociations;
+import org.apache.atlas.model.instance.AtlasEntityWithAssociations;
+import org.apache.atlas.model.instance.EntityMutationResponse;
+import org.apache.atlas.model.instance.EntityMutations.EntityOperation;
+import org.apache.atlas.model.lineage.AtlasLineageInfo;
+import org.apache.atlas.model.lineage.AtlasLineageInfo.LineageDirection;
+import org.apache.atlas.model.lineage.AtlasLineageInfo.LineageRelation;
+import org.apache.atlas.model.typedef.AtlasBaseTypeDef;
+import org.apache.atlas.model.typedef.AtlasClassificationDef;
+import org.apache.atlas.model.typedef.AtlasEntityDef;
+import org.apache.atlas.model.typedef.AtlasEnumDef;
+import org.apache.atlas.model.typedef.AtlasStructDef;
+import org.apache.atlas.model.typedef.AtlasTypesDef;
+import org.apache.atlas.type.AtlasTypeUtil;
+import org.apache.atlas.utils.AuthenticationUtil;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.commons.configuration.Configuration;
+import org.apache.commons.lang.ArrayUtils;
+
+import javax.ws.rs.core.MultivaluedMap;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import static org.apache.atlas.model.typedef.AtlasStructDef.AtlasConstraintDef.CONSTRAINT_PARAM_ON_DELETE;
+import static org.apache.atlas.model.typedef.AtlasStructDef.AtlasConstraintDef.CONSTRAINT_PARAM_REF_ATTRIBUTE;
+import static org.apache.atlas.model.typedef.AtlasStructDef.AtlasConstraintDef.CONSTRAINT_PARAM_VAL_CASCADE;
+import static org.apache.atlas.model.typedef.AtlasStructDef.AtlasConstraintDef.CONSTRAINT_TYPE_FOREIGN_KEY;
+import static org.apache.atlas.model.typedef.AtlasStructDef.AtlasConstraintDef.CONSTRAINT_TYPE_MAPPED_FROM_REF;
+
+/**
+ * A driver that sets up sample types and entities using v2 types and entity model for testing purposes.
+ */
+public class QuickStartV2 {
+    public static final String ATLAS_REST_ADDRESS          = "atlas.rest.address";
+
+    public static final String SALES_DB                    = "Sales";
+    public static final String REPORTING_DB                = "Reporting";
+    public static final String LOGGING_DB                  = "Logging";
+
+    public static final String SALES_FACT_TABLE            = "sales_fact";
+    public static final String PRODUCT_DIM_TABLE           = "product_dim";
+    public static final String CUSTOMER_DIM_TABLE          = "customer_dim";
+    public static final String TIME_DIM_TABLE              = "time_dim";
+    public static final String SALES_FACT_DAILY_MV_TABLE   = "sales_fact_daily_mv";
+    public static final String SALES_FACT_MONTHLY_MV_TABLE = "sales_fact_monthly_mv";
+    public static final String LOG_FACT_DAILY_MV_TABLE     = "log_fact_daily_mv";
+    public static final String LOG_FACT_MONTHLY_MV_TABLE   = "logging_fact_monthly_mv";
+
+    public static final String TIME_ID_COLUMN              = "time_id";
+    public static final String PRODUCT_ID_COLUMN           = "product_id";
+    public static final String CUSTOMER_ID_COLUMN          = "customer_id";
+    public static final String APP_ID_COLUMN               = "app_id";
+    public static final String MACHINE_ID_COLUMN           = "machine_id";
+    public static final String PRODUCT_NAME_COLUMN         = "product_name";
+    public static final String BRAND_NAME_COLUMN           = "brand_name";
+    public static final String NAME_COLUMN                 = "name";
+    public static final String SALES_COLUMN                = "sales";
+    public static final String LOG_COLUMN                  = "log";
+    public static final String ADDRESS_COLUMN              = "address";
+    public static final String DAY_OF_YEAR_COLUMN          = "dayOfYear";
+    public static final String WEEKDAY_COLUMN              = "weekDay";
+
+    public static final String DIMENSION_CLASSIFICATION    = "Dimension";
+    public static final String FACT_CLASSIFICATION         = "Fact";
+    public static final String PII_CLASSIFICATION          = "PII";
+    public static final String METRIC_CLASSIFICATION       = "Metric";
+    public static final String ETL_CLASSIFICATION          = "ETL";
+    public static final String JDBC_CLASSIFICATION         = "JdbcAccess";
+    public static final String LOGDATA_CLASSIFICATION      = "Log Data";
+
+    public static final String LOAD_SALES_DAILY_PROCESS    = "loadSalesDaily";
+    public static final String LOAD_SALES_MONTHLY_PROCESS  = "loadSalesMonthly";
+    public static final String LOAD_LOGS_MONTHLY_PROCESS   = "loadLogsMonthly";
+
+    public static final String PRODUCT_DIM_VIEW            = "product_dim_view";
+    public static final String CUSTOMER_DIM_VIEW           = "customer_dim_view";
+
+    public static final String DATABASE_TYPE               = "DB";
+    public static final String COLUMN_TYPE                 = "Column";
+    public static final String TABLE_TYPE                  = "Table";
+    public static final String VIEW_TYPE                   = "View";
+    public static final String LOAD_PROCESS_TYPE           = "LoadProcess";
+    public static final String STORAGE_DESC_TYPE           = "StorageDesc";
+
+    public static final String[] TYPES = { DATABASE_TYPE, TABLE_TYPE, STORAGE_DESC_TYPE, COLUMN_TYPE, LOAD_PROCESS_TYPE,
+                                           VIEW_TYPE, JDBC_CLASSIFICATION, ETL_CLASSIFICATION, METRIC_CLASSIFICATION,
+                                           PII_CLASSIFICATION, FACT_CLASSIFICATION, DIMENSION_CLASSIFICATION, LOGDATA_CLASSIFICATION };
+
+    public static void main(String[] args) throws Exception {
+        String[] basicAuthUsernamePassword = null;
+
+        if (!AuthenticationUtil.isKerberosAuthenticationEnabled()) {
+            basicAuthUsernamePassword = AuthenticationUtil.getBasicAuthenticationInput();
+        }
+
+        runQuickstart(args, basicAuthUsernamePassword);
+    }
+
+    @VisibleForTesting
+    static void runQuickstart(String[] args, String[] basicAuthUsernamePassword) throws Exception {
+        String[] urls = getServerUrl(args);
+        QuickStartV2 quickStartV2;
+
+        if (!AuthenticationUtil.isKerberosAuthenticationEnabled()) {
+            quickStartV2 = new QuickStartV2(urls, basicAuthUsernamePassword);
+        } else {
+            quickStartV2 = new QuickStartV2(urls);
+        }
+
+        // Shows how to create v2 types in Atlas for your meta model
+        quickStartV2.createTypes();
+
+        // Shows how to create v2 entities (instances) for the added types in Atlas
+        quickStartV2.createEntities();
+
+        // Shows some search queries using DSL based on types
+        quickStartV2.search();
+
+        // Shows some lineage information on entity
+        quickStartV2.lineage();
+        
+    }
+
+    static String[] getServerUrl(String[] args) throws AtlasException {
+        if (args.length > 0) {
+            return args[0].split(",");
+        }
+
+        Configuration configuration = ApplicationProperties.get();
+        String[] urls = configuration.getStringArray(ATLAS_REST_ADDRESS);
+
+        if (ArrayUtils.isEmpty(urls)) {
+            System.out.println("org.apache.atlas.examples.QuickStartV2 <Atlas REST address <http/https>://<atlas-fqdn>:<atlas-port> like http://localhost:21000>");
+            System.exit(-1);
+        }
+
+        return urls;
+    }
+
+    private final AtlasTypedefClientV2   typesClient;
+    private final AtlasEntitiesClientV2  entitiesClient;
+    private final AtlasDiscoveryClientV2 discoveryClient;
+    private final AtlasLineageClientV2   lineageClient;
+
+    QuickStartV2(String[] urls, String[] basicAuthUsernamePassword) {
+        typesClient     = new AtlasTypedefClientV2(urls,basicAuthUsernamePassword);
+        entitiesClient  = new AtlasEntitiesClientV2(urls,basicAuthUsernamePassword);
+        discoveryClient = new AtlasDiscoveryClientV2(urls,basicAuthUsernamePassword);
+        lineageClient   = new AtlasLineageClientV2(urls,basicAuthUsernamePassword);
+    }
+
+    QuickStartV2(String[] urls) throws AtlasException {
+        typesClient     = new AtlasTypedefClientV2(urls);
+        entitiesClient  = new AtlasEntitiesClientV2(urls);
+        discoveryClient = new AtlasDiscoveryClientV2(urls);
+        lineageClient   = new AtlasLineageClientV2(urls);
+    }
+
+
+    void createTypes() throws Exception {
+        AtlasTypesDef atlasTypesDef = createTypeDefinitions();
+
+        System.out.println("\nCreating sample types: ");
+        typesClient.createAtlasTypeDefs(atlasTypesDef);
+
+        verifyTypesCreated();
+    }
+
+    AtlasTypesDef createTypeDefinitions() throws Exception {
+        AtlasEntityDef dbType   = AtlasTypeUtil.createClassTypeDef(DATABASE_TYPE, DATABASE_TYPE, "1.0", null,
+                                  AtlasTypeUtil.createUniqueRequiredAttrDef("name", "string"),
+                                  AtlasTypeUtil.createOptionalAttrDef("description", "string"),
+                                  AtlasTypeUtil.createOptionalAttrDef("locationUri", "string"),
+                                  AtlasTypeUtil.createOptionalAttrDef("owner", "string"),
+                                  AtlasTypeUtil.createOptionalAttrDef("createTime", "long"));
+
+        AtlasEntityDef sdType   = AtlasTypeUtil.createClassTypeDef(STORAGE_DESC_TYPE, STORAGE_DESC_TYPE, "1.0", null,
+                                  AtlasTypeUtil.createOptionalAttrDefWithConstraint("table", TABLE_TYPE, CONSTRAINT_TYPE_FOREIGN_KEY,
+                                          new HashMap<String, Object>() {{ put(CONSTRAINT_PARAM_ON_DELETE, CONSTRAINT_PARAM_VAL_CASCADE); }}),
+                                  AtlasTypeUtil.createOptionalAttrDef("location", "string"),
+                                  AtlasTypeUtil.createOptionalAttrDef("inputFormat", "string"),
+                                  AtlasTypeUtil.createOptionalAttrDef("outputFormat", "string"),
+                                  AtlasTypeUtil.createRequiredAttrDef("compressed", "boolean"));
+
+        AtlasEntityDef colType  = AtlasTypeUtil.createClassTypeDef(COLUMN_TYPE, COLUMN_TYPE, "1.0", null,
+                                  AtlasTypeUtil.createOptionalAttrDef("name", "string"),
+                                  AtlasTypeUtil.createOptionalAttrDef("dataType", "string"),
+                                  AtlasTypeUtil.createOptionalAttrDef("comment", "string"),
+                                  AtlasTypeUtil.createOptionalAttrDefWithConstraint("table", TABLE_TYPE, CONSTRAINT_TYPE_FOREIGN_KEY,
+                                          new HashMap<String, Object>() {{ put(CONSTRAINT_PARAM_ON_DELETE, CONSTRAINT_PARAM_VAL_CASCADE); }}));
+
+        AtlasEntityDef tblType  = AtlasTypeUtil.createClassTypeDef(TABLE_TYPE, TABLE_TYPE, "1.0", ImmutableSet.of("DataSet"),
+                                  AtlasTypeUtil.createRequiredAttrDef("db", DATABASE_TYPE),
+                                  AtlasTypeUtil.createRequiredAttrDefWithConstraint("sd", STORAGE_DESC_TYPE, CONSTRAINT_TYPE_MAPPED_FROM_REF,
+                                          new HashMap<String, Object>() {{ put(CONSTRAINT_PARAM_REF_ATTRIBUTE, "table"); }}),
+                                  AtlasTypeUtil.createOptionalAttrDef("owner", "string"),
+                                  AtlasTypeUtil.createOptionalAttrDef("createTime", "long"),
+                                  AtlasTypeUtil.createOptionalAttrDef("lastAccessTime", "long"),
+                                  AtlasTypeUtil.createOptionalAttrDef("retention", "long"),
+                                  AtlasTypeUtil.createOptionalAttrDef("viewOriginalText", "string"),
+                                  AtlasTypeUtil.createOptionalAttrDef("viewExpandedText", "string"),
+                                  AtlasTypeUtil.createOptionalAttrDef("tableType", "string"),
+                                  AtlasTypeUtil.createOptionalAttrDef("temporary", "boolean"),
+                                  AtlasTypeUtil.createRequiredListAttrDefWithConstraint("columns", AtlasBaseTypeDef.getArrayTypeName(COLUMN_TYPE),
+                                          CONSTRAINT_TYPE_MAPPED_FROM_REF, new HashMap<String, Object>() {{ put(CONSTRAINT_PARAM_REF_ATTRIBUTE, "table"); }}));
+
+        AtlasEntityDef procType = AtlasTypeUtil.createClassTypeDef(LOAD_PROCESS_TYPE, LOAD_PROCESS_TYPE, "1.0", ImmutableSet.of("Process"),
+                                  AtlasTypeUtil.createOptionalAttrDef("userName", "string"),
+                                  AtlasTypeUtil.createOptionalAttrDef("startTime", "long"),
+                                  AtlasTypeUtil.createOptionalAttrDef("endTime", "long"),
+                                  AtlasTypeUtil.createRequiredAttrDef("queryText", "string"),
+                                  AtlasTypeUtil.createRequiredAttrDef("queryPlan", "string"),
+                                  AtlasTypeUtil.createRequiredAttrDef("queryId", "string"),
+                                  AtlasTypeUtil.createRequiredAttrDef("queryGraph", "string"));
+
+        AtlasEntityDef viewType = AtlasTypeUtil.createClassTypeDef(VIEW_TYPE, VIEW_TYPE, "1.0", ImmutableSet.of("DataSet"),
+                                  AtlasTypeUtil.createRequiredAttrDef("db", DATABASE_TYPE),
+                                  AtlasTypeUtil.createOptionalListAttrDef("inputTables", AtlasBaseTypeDef.getArrayTypeName(TABLE_TYPE)));
+
+        AtlasClassificationDef dimClassifDef    = AtlasTypeUtil.createTraitTypeDef(DIMENSION_CLASSIFICATION,  "Dimension Classification", "1.0", ImmutableSet.<String>of());
+        AtlasClassificationDef factClassifDef   = AtlasTypeUtil.createTraitTypeDef(FACT_CLASSIFICATION, "Fact Classification", "1.0", ImmutableSet.<String>of());
+        AtlasClassificationDef piiClassifDef    = AtlasTypeUtil.createTraitTypeDef(PII_CLASSIFICATION, "PII Classification", "1.0", ImmutableSet.<String>of());
+        AtlasClassificationDef metricClassifDef = AtlasTypeUtil.createTraitTypeDef(METRIC_CLASSIFICATION, "Metric Classification", "1.0", ImmutableSet.<String>of());
+        AtlasClassificationDef etlClassifDef    = AtlasTypeUtil.createTraitTypeDef(ETL_CLASSIFICATION, "ETL Classification", "1.0", ImmutableSet.<String>of());
+        AtlasClassificationDef jdbcClassifDef   = AtlasTypeUtil.createTraitTypeDef(JDBC_CLASSIFICATION, "JdbcAccess Classification", "1.0", ImmutableSet.<String>of());
+        AtlasClassificationDef logClassifDef    = AtlasTypeUtil.createTraitTypeDef(LOGDATA_CLASSIFICATION, "LogData Classification", "1.0", ImmutableSet.<String>of());
+
+        return AtlasTypeUtil.getTypesDef(ImmutableList.<AtlasEnumDef>of(),
+                                         ImmutableList.<AtlasStructDef>of(),
+                                         ImmutableList.of(dimClassifDef, factClassifDef, piiClassifDef, metricClassifDef, etlClassifDef, jdbcClassifDef, logClassifDef),
+                                         ImmutableList.of(dbType, sdType, colType, tblType, procType, viewType));
+    }
+
+    void createEntities() throws Exception {
+        System.out.println("\nCreating sample entities: ");
+
+        // Database entities
+        AtlasEntity salesDB     = createDatabase(SALES_DB, "sales database", "John ETL", "hdfs://host:8000/apps/warehouse/sales");
+        AtlasEntity reportingDB = createDatabase(REPORTING_DB, "reporting database", "Jane BI", "hdfs://host:8000/apps/warehouse/reporting");
+        AtlasEntity logDB       = createDatabase(LOGGING_DB, "logging database", "Tim ETL", "hdfs://host:8000/apps/warehouse/logging");
+
+        // Storage Descriptor entities
+        AtlasEntity storageDesc = createStorageDescriptor("hdfs://host:8000/apps/warehouse/sales", "TextInputFormat", "TextOutputFormat", true);
+
+        // Column entities
+        List<AtlasEntity> salesFactColumns   = ImmutableList.of(createColumn(TIME_ID_COLUMN, "int", "time id"),
+                                                                createColumn(PRODUCT_ID_COLUMN, "int", "product id"),
+                                                                createColumn(CUSTOMER_ID_COLUMN, "int", "customer id", PII_CLASSIFICATION),
+                                                                createColumn(SALES_COLUMN, "double", "product id", METRIC_CLASSIFICATION));
+
+        List<AtlasEntity> logFactColumns     = ImmutableList.of(createColumn(TIME_ID_COLUMN, "int", "time id"),
+                                                                createColumn(APP_ID_COLUMN, "int", "app id"),
+                                                                createColumn(MACHINE_ID_COLUMN, "int", "machine id"),
+                                                                createColumn(LOG_COLUMN, "string", "log data", LOGDATA_CLASSIFICATION));
+
+        List<AtlasEntity> productDimColumns  = ImmutableList.of(createColumn(PRODUCT_ID_COLUMN, "int", "product id"),
+                                                                createColumn(PRODUCT_NAME_COLUMN, "string", "product name"),
+                                                                createColumn(BRAND_NAME_COLUMN, "int", "brand name"));
+
+        List<AtlasEntity> timeDimColumns     = ImmutableList.of(createColumn(TIME_ID_COLUMN, "int", "time id"),
+                                                                createColumn(DAY_OF_YEAR_COLUMN, "int", "day Of Year"),
+                                                                createColumn(WEEKDAY_COLUMN, "int", "week Day"));
+
+        List<AtlasEntity> customerDimColumns = ImmutableList.of(createColumn(CUSTOMER_ID_COLUMN, "int", "customer id", PII_CLASSIFICATION),
+                                                                createColumn(NAME_COLUMN, "string", "customer name", PII_CLASSIFICATION),
+                                                                createColumn(ADDRESS_COLUMN, "string", "customer address", PII_CLASSIFICATION));
+
+        // Table entities
+        AtlasEntity salesFact          = createTable(SALES_FACT_TABLE, "sales fact table", salesDB, storageDesc,
+                                                     "Joe", "Managed", salesFactColumns, FACT_CLASSIFICATION);
+        AtlasEntity productDim         = createTable(PRODUCT_DIM_TABLE, "product dimension table", salesDB, storageDesc,
+                                                     "John Doe", "Managed", productDimColumns, DIMENSION_CLASSIFICATION);
+        AtlasEntity customerDim        = createTable(CUSTOMER_DIM_TABLE, "customer dimension table", salesDB, storageDesc,
+                                                     "fetl", "External", customerDimColumns, DIMENSION_CLASSIFICATION);
+        AtlasEntity timeDim            = createTable(TIME_DIM_TABLE, "time dimension table", salesDB, storageDesc,
+                                                     "John Doe", "External", timeDimColumns, DIMENSION_CLASSIFICATION);
+        AtlasEntity loggingFactDaily   = createTable(LOG_FACT_DAILY_MV_TABLE, "log fact daily materialized view", logDB,
+                                                     storageDesc, "Tim ETL", "Managed", logFactColumns, LOGDATA_CLASSIFICATION);
+        AtlasEntity loggingFactMonthly = createTable(LOG_FACT_MONTHLY_MV_TABLE, "logging fact monthly materialized view", logDB,
+                                                     storageDesc, "Tim ETL", "Managed", logFactColumns, LOGDATA_CLASSIFICATION);
+        AtlasEntity salesFactDaily     = createTable(SALES_FACT_DAILY_MV_TABLE, "sales fact daily materialized view", reportingDB,
+                                                     storageDesc, "Joe BI", "Managed", salesFactColumns, METRIC_CLASSIFICATION);
+        AtlasEntity salesFactMonthly   = createTable(SALES_FACT_MONTHLY_MV_TABLE, "sales fact monthly materialized view", reportingDB,
+                                                     storageDesc, "Jane BI", "Managed", salesFactColumns, METRIC_CLASSIFICATION);
+
+        // View entities
+        createView(PRODUCT_DIM_VIEW, reportingDB, ImmutableList.of(productDim), DIMENSION_CLASSIFICATION, JDBC_CLASSIFICATION);
+        createView(CUSTOMER_DIM_VIEW, reportingDB, ImmutableList.of(customerDim), DIMENSION_CLASSIFICATION, JDBC_CLASSIFICATION);
+
+        // Process entities
+        createProcess(LOAD_SALES_DAILY_PROCESS, "hive query for daily summary", "John ETL",
+                      ImmutableList.of(salesFact, timeDim),
+                      ImmutableList.of(salesFactDaily),
+                      "create table as select ", "plan", "id", "graph", ETL_CLASSIFICATION);
+
+        createProcess(LOAD_SALES_MONTHLY_PROCESS, "hive query for monthly summary", "John ETL",
+                      ImmutableList.of(salesFactDaily),
+                      ImmutableList.of(salesFactMonthly),
+                      "create table as select ", "plan", "id", "graph", ETL_CLASSIFICATION);
+
+        createProcess(LOAD_LOGS_MONTHLY_PROCESS, "hive query for monthly summary", "Tim ETL",
+                      ImmutableList.of(loggingFactDaily),
+                      ImmutableList.of(loggingFactMonthly),
+                      "create table as select ", "plan", "id", "graph", ETL_CLASSIFICATION);
+    }
+
+    private AtlasEntity createInstance(AtlasEntity entity, String[] traitNames) throws Exception {
+        AtlasEntity ret = null;
+        EntityMutationResponse  response = entitiesClient.createEntity(entity);
+        List<AtlasEntityHeader> entities = response.getEntitiesByOperation(EntityOperation.CREATE);
+
+        if (CollectionUtils.isNotEmpty(entities)) {
+            ret = entitiesClient.getEntityByGuid(entities.get(0).getGuid());
+            System.out.println("Created entity of type [" + ret.getTypeName() + "], guid: " + ret.getGuid());
+        }
+
+        if (ArrayUtils.isNotEmpty(traitNames)) {
+            entitiesClient.addClassifications(ret.getGuid(), toAtlasClassifications(traitNames));
+        }
+
+        return ret;
+    }
+
+    AtlasEntity createDatabase(String name, String description, String owner, String locationUri, String... traitNames)
+            throws Exception {
+        AtlasEntityWithAssociations entity = new AtlasEntityWithAssociations(DATABASE_TYPE);
+
+        entity.setClassifications(toAtlasClassifications(traitNames));
+        entity.setAttribute("name", name);
+        entity.setAttribute("description", description);
+        entity.setAttribute("owner", owner);
+        entity.setAttribute("locationuri", locationUri);
+        entity.setAttribute("createTime", System.currentTimeMillis());
+
+        return createInstance(entity, traitNames);
+    }
+
+    private List<AtlasClassification> toAtlasClassifications(String[] traitNames) {
+        List<AtlasClassification> ret    = new ArrayList<>();
+        ImmutableList<String>     traits = ImmutableList.copyOf(traitNames);
+
+        if (CollectionUtils.isNotEmpty(traits)) {
+            for (String trait : traits) {
+                ret.add(new AtlasClassification(trait));
+            }
+        }
+
+        return ret;
+    }
+
+    AtlasEntity createStorageDescriptor(String location, String inputFormat, String outputFormat, boolean compressed)
+            throws Exception {
+        AtlasEntity entity = new AtlasEntity(STORAGE_DESC_TYPE);
+
+        entity.setAttribute("location", location);
+        entity.setAttribute("inputFormat", inputFormat);
+        entity.setAttribute("outputFormat", outputFormat);
+        entity.setAttribute("compressed", compressed);
+
+        return createInstance(entity, null);
+    }
+
+    AtlasEntity createColumn(String name, String dataType, String comment, String... traitNames) throws Exception {
+
+        AtlasEntityWithAssociations entity = new AtlasEntityWithAssociations(COLUMN_TYPE);
+        entity.setClassifications(toAtlasClassifications(traitNames));
+        entity.setAttribute("name", name);
+        entity.setAttribute("dataType", dataType);
+        entity.setAttribute("comment", comment);
+
+        return createInstance(entity, traitNames);
+    }
+
+    AtlasEntity createTable(String name, String description, AtlasEntity db, AtlasEntity sd, String owner, String tableType,
+                            List<AtlasEntity> columns, String... traitNames) throws Exception {
+        AtlasEntityWithAssociations entity = new AtlasEntityWithAssociations(TABLE_TYPE);
+
+        entity.setClassifications(toAtlasClassifications(traitNames));
+        entity.setAttribute("name", name);
+        entity.setAttribute(AtlasClient.REFERENCEABLE_ATTRIBUTE_NAME, name);
+        entity.setAttribute("description", description);
+        entity.setAttribute("owner", owner);
+        entity.setAttribute("tableType", tableType);
+        entity.setAttribute("createTime", System.currentTimeMillis());
+        entity.setAttribute("lastAccessTime", System.currentTimeMillis());
+        entity.setAttribute("retention", System.currentTimeMillis());
+        entity.setAttribute("db", db);
+        entity.setAttribute("sd", sd);
+        entity.setAttribute("columns", columns);
+
+        return createInstance(entity, traitNames);
+    }
+
+    AtlasEntity createProcess(String name, String description, String user, List<AtlasEntity> inputs, List<AtlasEntity> outputs,
+            String queryText, String queryPlan, String queryId, String queryGraph, String... traitNames) throws Exception {
+        AtlasEntityWithAssociations entity = new AtlasEntityWithAssociations(LOAD_PROCESS_TYPE);
+
+        entity.setClassifications(toAtlasClassifications(traitNames));
+        entity.setAttribute(AtlasClient.NAME, name);
+        entity.setAttribute(AtlasClient.REFERENCEABLE_ATTRIBUTE_NAME, name);
+        entity.setAttribute("description", description);
+        entity.setAttribute("inputs", inputs);
+        entity.setAttribute("outputs", outputs);
+        entity.setAttribute("user", user);
+        entity.setAttribute("startTime", System.currentTimeMillis());
+        entity.setAttribute("endTime", System.currentTimeMillis() + 10000);
+        entity.setAttribute("queryText", queryText);
+        entity.setAttribute("queryPlan", queryPlan);
+        entity.setAttribute("queryId", queryId);
+        entity.setAttribute("queryGraph", queryGraph);
+
+        return createInstance(entity, traitNames);
+    }
+
+    AtlasEntity createView(String name, AtlasEntity db, List<AtlasEntity> inputTables, String... traitNames) throws Exception {
+        AtlasEntityWithAssociations entity = new AtlasEntityWithAssociations(VIEW_TYPE);
+
+        entity.setClassifications(toAtlasClassifications(traitNames));
+        entity.setAttribute("name", name);
+        entity.setAttribute(AtlasClient.REFERENCEABLE_ATTRIBUTE_NAME, name);
+        entity.setAttribute("db", db);
+        entity.setAttribute("inputTables", inputTables);
+
+        return createInstance(entity, traitNames);
+    }
+
+    private void verifyTypesCreated() throws Exception {
+        MultivaluedMap<String, String> searchParams = new MultivaluedMapImpl();
+
+        for (String typeName : TYPES) {
+            searchParams.clear();
+            searchParams.add(SearchFilter.PARAM_NAME, typeName);
+            SearchFilter searchFilter = new SearchFilter(searchParams);
+            AtlasTypesDef searchDefs = typesClient.getAllTypeDefs(searchFilter);
+
+            assert (!searchDefs.isEmpty());
+            System.out.println("Created type [" + typeName + "]");
+        }
+    }
+
+    private String[] getDSLQueries() {
+        return new String[]{
+                "from DB",
+                "DB",
+                "DB where name=\"Reporting\"",
+                "DB where DB.name=\"Reporting\"",
+                "DB name = \"Reporting\"",
+                "DB DB.name = \"Reporting\"",
+                "DB where name=\"Reporting\" select name, owner",
+                "DB where DB.name=\"Reporting\" select name, owner",
+                "DB has name",
+                "DB where DB has name",
+                "DB, Table",
+                "DB is JdbcAccess",
+                "from Table",
+                "Table",
+                "Table is Dimension",
+                "Column where Column isa PII",
+                "View is Dimension",
+                "Column select Column.name",
+                "Column select name",
+                "Column where Column.name=\"customer_id\"",
+                "from Table select Table.name",
+                "DB where (name = \"Reporting\")",
+                "DB where (name = \"Reporting\") select name as _col_0, owner as _col_1",
+                "DB where DB is JdbcAccess",
+                "DB where DB has name",
+                "DB Table",
+                "DB as db1 Table where (db1.name = \"Reporting\")",
+                "DB where (name = \"Reporting\") select name as _col_0, (createTime + 1) as _col_1 ",
+                DIMENSION_CLASSIFICATION,
+                JDBC_CLASSIFICATION,
+                ETL_CLASSIFICATION,
+                METRIC_CLASSIFICATION,
+                PII_CLASSIFICATION,
+                "`Log Data`",
+                "Table where name=\"sales_fact\", columns",
+                "Table where name=\"sales_fact\", columns as column select column.name, column.dataType, column.comment",
+                "from DataSet",
+                "from Process" };
+    }
+
+    private void search() throws Exception {
+        System.out.println("\nSample DSL Queries: ");
+
+        for (String dslQuery : getDSLQueries()) {
+            AtlasSearchResult results = discoveryClient.dslSearchWithParams(dslQuery, 10, 0);
+
+            if (results != null) {
+                List<AtlasEntityHeaderWithAssociations> entitiesResult  = results.getEntities();
+                List<AtlasFullTextResult> fullTextResults = results.getFullTextResult();
+                AttributeSearchResult     attribResult    = results.getAttributes();
+
+                if (CollectionUtils.isNotEmpty(entitiesResult)) {
+                    System.out.println("query [" + dslQuery + "] returned [" + entitiesResult.size() + "] rows.");
+                } else if (CollectionUtils.isNotEmpty(fullTextResults)) {
+                    System.out.println("query [" + dslQuery + "] returned [" + fullTextResults.size() + "] rows.");
+                } else if (attribResult != null) {
+                    System.out.println("query [" + dslQuery + "] returned [" + attribResult.getValues().size() + "] rows.");
+                }
+            } else {
+                System.out.println("query [" + dslQuery + "] failed, results:" + results);
+            }
+        }
+    }
+
+    private void lineage() throws AtlasServiceException {
+        System.out.println("\nSample Lineage Info: ");
+
+        AtlasLineageInfo lineageInfo = lineageClient.getLineageInfo(getTableId(SALES_FACT_DAILY_MV_TABLE), LineageDirection.BOTH, 0);
+        Set<LineageRelation> relations = lineageInfo.getRelations();
+        Map<String, AtlasEntityHeader> guidEntityMap = lineageInfo.getGuidEntityMap();
+
+        for (LineageRelation relation : relations) {
+            AtlasEntityHeader fromEntity = guidEntityMap.get(relation.getFromEntityId());
+            AtlasEntityHeader toEntity   = guidEntityMap.get(relation.getToEntityId());
+
+            System.out.println(fromEntity.getDisplayText() + "(" + fromEntity.getTypeName() + ") -> " +
+                               toEntity.getDisplayText()   + "(" + toEntity.getTypeName() + ")");
+        }
+    }
+
+    private String getTableId(String tableName) throws AtlasServiceException {
+        AtlasEntity tableEntity = entitiesClient.getEntityByAttribute(TABLE_TYPE, AtlasClient.REFERENCEABLE_ATTRIBUTE_NAME, tableName);
+        return tableEntity.getGuid();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/57f4f79d/webapp/src/test/java/org/apache/atlas/examples/QuickStartV2IT.java
----------------------------------------------------------------------
diff --git a/webapp/src/test/java/org/apache/atlas/examples/QuickStartV2IT.java b/webapp/src/test/java/org/apache/atlas/examples/QuickStartV2IT.java
new file mode 100644
index 0000000..9528139
--- /dev/null
+++ b/webapp/src/test/java/org/apache/atlas/examples/QuickStartV2IT.java
@@ -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.atlas.examples;
+
+import org.apache.atlas.AtlasClient;
+import org.apache.atlas.AtlasServiceException;
+import org.apache.atlas.model.instance.AtlasClassification;
+import org.apache.atlas.model.instance.AtlasEntity;
+import org.apache.atlas.model.instance.AtlasEntityHeader;
+import org.apache.atlas.model.lineage.AtlasLineageInfo;
+import org.apache.atlas.model.lineage.AtlasLineageInfo.LineageDirection;
+import org.apache.atlas.model.lineage.AtlasLineageInfo.LineageRelation;
+import org.apache.atlas.web.resources.BaseResourceIT;
+import org.codehaus.jettison.json.JSONException;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+import static org.testng.Assert.assertEquals;
+import static org.testng.Assert.assertNotNull;
+import static org.testng.AssertJUnit.assertTrue;
+
+public class QuickStartV2IT extends BaseResourceIT {
+
+    @BeforeClass
+    public void runQuickStart() throws Exception {
+        super.setUp();
+        QuickStartV2.runQuickstart(new String[]{}, new String[]{"admin", "admin"});
+    }
+
+    @Test
+    public void testDBIsAdded() throws Exception {
+        AtlasEntity db = getDB(QuickStartV2.SALES_DB);
+        Map<String, Object> dbAttributes = db.getAttributes();
+        assertEquals(QuickStartV2.SALES_DB, dbAttributes.get("name"));
+        assertEquals("sales database", dbAttributes.get("description"));
+    }
+
+    private AtlasEntity getDB(String dbName) throws AtlasServiceException, JSONException {
+        AtlasEntity dbEntity = entitiesClientV2.getEntityByAttribute(QuickStartV2.DATABASE_TYPE, "name", dbName);
+        return dbEntity;
+    }
+
+    @Test
+    public void testTablesAreAdded() throws AtlasServiceException, JSONException {
+        AtlasEntity table = getTable(QuickStart.SALES_FACT_TABLE);
+        verifySimpleTableAttributes(table);
+
+        verifyDBIsLinkedToTable(table);
+
+        verifyColumnsAreAddedToTable(table);
+
+        verifyTrait(table);
+    }
+
+    private AtlasEntity getTable(String tableName) throws AtlasServiceException {
+        AtlasEntity tableEntity = entitiesClientV2.getEntityByAttribute(QuickStartV2.TABLE_TYPE, AtlasClient.REFERENCEABLE_ATTRIBUTE_NAME, tableName);
+        return tableEntity;
+    }
+
+    private AtlasEntity getProcess(String processName) throws AtlasServiceException {
+        AtlasEntity processEntity = entitiesClientV2.getEntityByAttribute(QuickStartV2.LOAD_PROCESS_TYPE, AtlasClient.REFERENCEABLE_ATTRIBUTE_NAME, processName);
+        return processEntity;
+    }
+
+    private void verifyTrait(AtlasEntity table) throws AtlasServiceException {
+        AtlasClassification.AtlasClassifications classfications = entitiesClientV2.getClassifications(table.getGuid());
+        List<AtlasClassification> traits = classfications.getList();
+        assertNotNull(traits.get(0).getTypeName());
+    }
+
+    private void verifyColumnsAreAddedToTable(AtlasEntity table) throws JSONException {
+        Map<String, Object> tableAttributes = table.getAttributes();
+        List<AtlasEntity> columns = (List<AtlasEntity>) tableAttributes.get("columns");
+        assertEquals(4, columns.size());
+
+        Map<String, Object> column = (Map) columns.get(0);
+        Map<String, Object> columnAttributes = (Map) column.get("attributes");
+        assertEquals(QuickStartV2.TIME_ID_COLUMN, columnAttributes.get("name"));
+        assertEquals("int", columnAttributes.get("dataType"));
+    }
+
+    private void verifyDBIsLinkedToTable(AtlasEntity table) throws AtlasServiceException, JSONException {
+        AtlasEntity db = getDB(QuickStartV2.SALES_DB);
+        Map<String, Object> tableAttributes = table.getAttributes();
+        Map dbFromTable = (Map) tableAttributes.get("db");
+        assertEquals(db.getGuid(), dbFromTable.get("guid"));
+    }
+
+    private void verifySimpleTableAttributes(AtlasEntity table) throws JSONException {
+        Map<String, Object> tableAttributes = table.getAttributes();
+        assertEquals(QuickStartV2.SALES_FACT_TABLE, tableAttributes.get("name"));
+        assertEquals("sales fact table", tableAttributes.get("description"));
+    }
+
+    @Test
+    public void testProcessIsAdded() throws AtlasServiceException, JSONException {
+        AtlasEntity loadProcess = entitiesClientV2.getEntityByAttribute(QuickStartV2.LOAD_PROCESS_TYPE, AtlasClient.REFERENCEABLE_ATTRIBUTE_NAME,
+                QuickStartV2.LOAD_SALES_DAILY_PROCESS);
+
+        Map loadProcessAttribs = loadProcess.getAttributes();
+        assertEquals(QuickStartV2.LOAD_SALES_DAILY_PROCESS, loadProcessAttribs.get(AtlasClient.NAME));
+        assertEquals("hive query for daily summary", loadProcessAttribs.get("description"));
+
+        List inputs = (List) loadProcessAttribs.get("inputs");
+        List outputs = (List) loadProcessAttribs.get("outputs");
+        assertEquals(2, inputs.size());
+
+        String salesFactTableId = getTableId(QuickStartV2.SALES_FACT_TABLE);
+        String timeDimTableId = getTableId(QuickStartV2.TIME_DIM_TABLE);
+        String salesFactDailyMVId = getTableId(QuickStartV2.SALES_FACT_DAILY_MV_TABLE);
+
+        assertEquals(salesFactTableId, ((Map) inputs.get(0)).get("guid"));
+        assertEquals(timeDimTableId, ((Map) inputs.get(1)).get("guid"));
+        assertEquals(salesFactDailyMVId, ((Map) outputs.get(0)).get("guid"));
+    }
+
+    private String getTableId(String tableName) throws AtlasServiceException {
+        return getTable(tableName).getGuid();
+    }
+
+    private String getProcessId(String processName) throws AtlasServiceException {
+        return getProcess(processName).getGuid();
+    }
+
+    @Test
+    public void testLineageIsMaintained() throws AtlasServiceException, JSONException {
+        String salesFactTableId      = getTableId(QuickStartV2.SALES_FACT_TABLE);
+        String timeDimTableId        = getTableId(QuickStartV2.TIME_DIM_TABLE);
+        String salesFactDailyMVId    = getTableId(QuickStartV2.SALES_FACT_DAILY_MV_TABLE);
+        String salesFactMonthlyMvId  = getTableId(QuickStartV2.SALES_FACT_MONTHLY_MV_TABLE);
+        String salesDailyProcessId   = getProcessId(QuickStartV2.LOAD_SALES_DAILY_PROCESS);
+        String salesMonthlyProcessId = getProcessId(QuickStartV2.LOAD_SALES_MONTHLY_PROCESS);
+
+        AtlasLineageInfo inputLineage = lineageClientV2.getLineageInfo(salesFactDailyMVId, LineageDirection.BOTH, 0);
+        List<LineageRelation> relations = new ArrayList<>(inputLineage.getRelations());
+        Map<String, AtlasEntityHeader> entityMap = inputLineage.getGuidEntityMap();
+
+        assertEquals(relations.size(), 5);
+        assertEquals(entityMap.size(), 6);
+
+        assertTrue(entityMap.containsKey(salesFactTableId));
+        assertTrue(entityMap.containsKey(timeDimTableId));
+        assertTrue(entityMap.containsKey(salesFactDailyMVId));
+        assertTrue(entityMap.containsKey(salesDailyProcessId));
+        assertTrue(entityMap.containsKey(salesFactMonthlyMvId));
+        assertTrue(entityMap.containsKey(salesMonthlyProcessId));
+    }
+
+    @Test
+    public void testViewIsAdded() throws AtlasServiceException, JSONException {
+        AtlasEntity view = entitiesClientV2.getEntityByAttribute(QuickStartV2.VIEW_TYPE, AtlasClient.REFERENCEABLE_ATTRIBUTE_NAME, QuickStartV2.PRODUCT_DIM_VIEW);
+        Map<String, Object> viewAttributes = view.getAttributes();
+        assertEquals(QuickStartV2.PRODUCT_DIM_VIEW, viewAttributes.get(AtlasClient.NAME));
+
+        String productDimId = getTable(QuickStartV2.PRODUCT_DIM_TABLE).getGuid();
+        List inputTables = (List) viewAttributes.get("inputTables");
+        Map inputTablesMap = (Map) inputTables.get(0);
+        assertEquals(productDimId, inputTablesMap.get("guid"));
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/57f4f79d/webapp/src/test/java/org/apache/atlas/web/resources/BaseResourceIT.java
----------------------------------------------------------------------
diff --git a/webapp/src/test/java/org/apache/atlas/web/resources/BaseResourceIT.java b/webapp/src/test/java/org/apache/atlas/web/resources/BaseResourceIT.java
index 9ea71ad..18bbc3b 100755
--- a/webapp/src/test/java/org/apache/atlas/web/resources/BaseResourceIT.java
+++ b/webapp/src/test/java/org/apache/atlas/web/resources/BaseResourceIT.java
@@ -26,6 +26,7 @@ import org.apache.atlas.ApplicationProperties;
 import org.apache.atlas.AtlasClient;
 import org.apache.atlas.AtlasDiscoveryClientV2;
 import org.apache.atlas.AtlasEntitiesClientV2;
+import org.apache.atlas.AtlasLineageClientV2;
 import org.apache.atlas.AtlasServiceException;
 import org.apache.atlas.AtlasTypedefClientV2;
 import org.apache.atlas.model.instance.AtlasClassification;
@@ -87,6 +88,7 @@ public abstract class BaseResourceIT {
     protected AtlasTypedefClientV2 typedefClientV2;
     protected AtlasEntitiesClientV2 entitiesClientV2;
     protected AtlasDiscoveryClientV2 discoveryClientV2;
+    protected AtlasLineageClientV2 lineageClientV2;
 
     public static final Logger LOG = LoggerFactory.getLogger(BaseResourceIT.class);
     protected static final int MAX_WAIT_TIME = 60000;
@@ -113,11 +115,13 @@ public abstract class BaseResourceIT {
             typedefClientV2 = new AtlasTypedefClientV2(atlasUrls, new String[]{"admin", "admin"});
             entitiesClientV2 = new AtlasEntitiesClientV2(atlasUrls, new String[]{"admin", "admin"});
             discoveryClientV2 = new AtlasDiscoveryClientV2(atlasUrls, new String[]{"admin", "admin"});
+            lineageClientV2 = new AtlasLineageClientV2(atlasUrls, new String[]{"admin", "admin"});
         } else {
             atlasClientV1 = new AtlasClient(atlasUrls);
             typedefClientV2 = new AtlasTypedefClientV2(atlasUrls);
             entitiesClientV2 = new AtlasEntitiesClientV2(atlasUrls);
             discoveryClientV2 = new AtlasDiscoveryClientV2(atlasUrls);
+            lineageClientV2 = new AtlasLineageClientV2(atlasUrls);
         }
     }