Commit 15967a93 by Ashutosh Mestry

ATLAS-2555: Migration-Import: Support for BigInteger, BigDecimal. Unit tests.

parent 7515915f
......@@ -28,6 +28,7 @@ import javax.script.ScriptException;
import org.apache.atlas.exception.AtlasBaseException;
import org.apache.atlas.groovy.GroovyExpression;
import org.apache.atlas.model.impexp.MigrationStatus;
import org.apache.atlas.type.AtlasType;
/**
......@@ -320,4 +321,6 @@ public interface AtlasGraph<V, E> {
boolean isMultiProperty(String name);
void loadLegacyGraphSON(Map<String, String> relationshipCache, InputStream fs) throws AtlasBaseException;
MigrationStatus getMigrationStatus();
}
......@@ -25,6 +25,7 @@ import org.apache.atlas.AtlasErrorCode;
import org.apache.atlas.AtlasException;
import org.apache.atlas.exception.AtlasBaseException;
import org.apache.atlas.groovy.GroovyExpression;
import org.apache.atlas.model.impexp.MigrationStatus;
import org.apache.atlas.repository.graphdb.AtlasEdge;
import org.apache.atlas.repository.graphdb.AtlasGraph;
import org.apache.atlas.repository.graphdb.AtlasGraphManagement;
......@@ -33,6 +34,7 @@ import org.apache.atlas.repository.graphdb.AtlasIndexQuery;
import org.apache.atlas.repository.graphdb.AtlasSchemaViolationException;
import org.apache.atlas.repository.graphdb.AtlasVertex;
import org.apache.atlas.repository.graphdb.GremlinVersion;
import org.apache.atlas.repository.graphdb.janus.migration.ReaderStatusManager;
import org.apache.atlas.repository.graphdb.janus.query.AtlasJanusGraphQuery;
import org.apache.atlas.repository.graphdb.utils.IteratorToIterableAdapter;
import org.apache.atlas.type.AtlasType;
......@@ -459,4 +461,9 @@ public class AtlasJanusGraph implements AtlasGraph<AtlasJanusVertex, AtlasJanusE
public void loadLegacyGraphSON(Map<String, String> relationshipCache, InputStream fs) throws AtlasBaseException {
AtlasJanusGraphDatabase.loadLegacyGraphSON(relationshipCache, fs);
}
@Override
public MigrationStatus getMigrationStatus() {
return AtlasJanusGraphDatabase.getMigrationStatus();
}
}
......@@ -21,9 +21,11 @@ package org.apache.atlas.repository.graphdb.janus;
import org.apache.atlas.ApplicationProperties;
import org.apache.atlas.AtlasException;
import org.apache.atlas.exception.AtlasBaseException;
import org.apache.atlas.model.impexp.MigrationStatus;
import org.apache.atlas.repository.graphdb.AtlasGraph;
import org.apache.atlas.repository.graphdb.GraphDatabase;
import org.apache.atlas.repository.graphdb.janus.migration.AtlasGraphSONReader;
import org.apache.atlas.repository.graphdb.janus.migration.ReaderStatusManager;
import org.apache.atlas.repository.graphdb.janus.serializer.BigDecimalSerializer;
import org.apache.atlas.repository.graphdb.janus.serializer.BigIntegerSerializer;
import org.apache.atlas.repository.graphdb.janus.serializer.TypeCategorySerializer;
......@@ -259,4 +261,8 @@ public class AtlasJanusGraphDatabase implements GraphDatabase<AtlasJanusVertex,
LOG.info("Done! loadLegacyGraphSON.");
}
}
public static MigrationStatus getMigrationStatus() {
return ReaderStatusManager.get(getGraphInstance());
}
}
......@@ -103,6 +103,16 @@ public final class AtlasGraphSONReader {
processElement(parser, new JsonNodeParsers.ParseEdge(), startIndex);
break;
case GraphSONTokensTP2.VERTEX_COUNT:
parser.nextToken();
LOG.info("Vertex count: {}", parser.getLongValue());
break;
case GraphSONTokensTP2.EDGE_COUNT:
parser.nextToken();
LOG.info("Edge count: {}", parser.getLongValue());
break;
default:
throw new IllegalStateException(String.format("Unexpected token in GraphSON - %s", fieldName));
}
......
......@@ -36,10 +36,15 @@ public final class GraphSONTokensTP2 {
public static final String TYPE_LONG = "long";
public static final String TYPE_SHORT = "short";
public static final String TYPE_BYTE = "byte";
public static final String TYPE_BIG_DECIMAL = "bigdecimal";
public static final String TYPE_BIG_INTEGER = "biginteger";
public static final String TYPE_DATE = "date";
public static final String TYPE_UNKNOWN = "unknown";
public static final String VERTICES = "vertices";
public static final String EDGES = "edges";
public static final String MODE = "mode";
public static final String VERTEX_COUNT = "vertexCount";
public static final String EDGE_COUNT = "edgeCount";
private GraphSONTokensTP2() {
}
......
......@@ -20,6 +20,7 @@ package org.apache.atlas.repository.graphdb.janus.migration;
import com.google.common.annotations.VisibleForTesting;
import org.apache.atlas.repository.Constants;
import org.apache.atlas.type.AtlasBuiltInTypes;
import org.apache.commons.lang.StringUtils;
import org.apache.tinkerpop.gremlin.structure.Edge;
import org.apache.tinkerpop.gremlin.structure.Graph;
......@@ -43,6 +44,8 @@ class GraphSONUtility {
private static final String EMPTY_STRING = "";
private final RelationshipTypeCache relationshipTypeCache;
private static AtlasBuiltInTypes.AtlasBigIntegerType bigIntegerType = new AtlasBuiltInTypes.AtlasBigIntegerType();
private static AtlasBuiltInTypes.AtlasBigDecimalType bigDecimalType = new AtlasBuiltInTypes.AtlasBigDecimalType();
public GraphSONUtility(final RelationshipTypeCache relationshipTypeCache) {
this.relationshipTypeCache = relationshipTypeCache;
......@@ -187,9 +190,11 @@ class GraphSONUtility {
if (StringUtils.isNotEmpty(typeName)) {
props.put(Constants.ENTITY_TYPE_PROPERTY_KEY, typeName);
} else {
if (LOG.isDebugEnabled()) {
LOG.debug("Could not find relationship type for: {}", edgeLabel);
}
}
}
private void addMandatoryRelationshipProperties(Map<String, Object> props) {
props.put(Constants.RELATIONSHIP_GUID_PROPERTY_KEY, UUID.randomUUID().toString());
......@@ -223,6 +228,7 @@ class GraphSONUtility {
}
}
@VisibleForTesting
static Map<String, Object> readProperties(final JsonNode node) {
final Map<String, Object> map = new HashMap<>();
final Iterator<Map.Entry<String, JsonNode>> iterator = node.fields();
......@@ -267,7 +273,13 @@ class GraphSONUtility {
} else if (node.get(GraphSONTokensTP2.TYPE).textValue().equals(GraphSONTokensTP2.TYPE_INTEGER)) {
propertyValue = node.get(GraphSONTokensTP2.VALUE).intValue();
} else if (node.get(GraphSONTokensTP2.TYPE).textValue().equals(GraphSONTokensTP2.TYPE_LONG)) {
propertyValue = node.get(GraphSONTokensTP2.VALUE).longValue();
propertyValue = node.get(GraphSONTokensTP2.VALUE).asLong();
} else if (node.get(GraphSONTokensTP2.TYPE).textValue().equals(GraphSONTokensTP2.TYPE_BIG_DECIMAL)) {
propertyValue = bigDecimalType.getNormalizedValue(node.get(GraphSONTokensTP2.VALUE));
} else if (node.get(GraphSONTokensTP2.TYPE).textValue().equals(GraphSONTokensTP2.TYPE_BIG_INTEGER)) {
propertyValue = bigIntegerType.getNormalizedValue(node.get(GraphSONTokensTP2.VALUE));
} else if (node.get(GraphSONTokensTP2.TYPE).textValue().equals(GraphSONTokensTP2.TYPE_DATE)) {
propertyValue = new Date(node.get(GraphSONTokensTP2.VALUE).asLong());
} else if (node.get(GraphSONTokensTP2.TYPE).textValue().equals(GraphSONTokensTP2.TYPE_STRING)) {
propertyValue = node.get(GraphSONTokensTP2.VALUE).textValue();
} else if (node.get(GraphSONTokensTP2.TYPE).textValue().equals(GraphSONTokensTP2.TYPE_LIST)) {
......@@ -308,6 +320,10 @@ class GraphSONUtility {
theValue = node.longValue();
} else if (node.isTextual()) {
theValue = node.textValue();
} else if (node.isBigDecimal()) {
theValue = node.decimalValue();
} else if (node.isBigInteger()) {
theValue = node.bigIntegerValue();
} else if (node.isArray()) {
// this is an array so just send it back so that it can be
// reprocessed to its primitive components
......
......@@ -73,42 +73,10 @@ public class JsonNodeParsers {
return el;
}
static Object getTypedValueFromJsonNode(final JsonNode node) {
Object theValue = null;
if (node != null && !node.isNull()) {
if (node.isBoolean()) {
theValue = node.booleanValue();
} else if (node.isDouble()) {
theValue = node.doubleValue();
} else if (node.isFloatingPointNumber()) {
theValue = node.floatValue();
} else if (node.isInt()) {
theValue = node.intValue();
} else if (node.isLong()) {
theValue = node.longValue();
} else if (node.isTextual()) {
theValue = node.textValue();
} else if (node.isArray()) {
// this is an array so just send it back so that it can be
// reprocessed to its primitive components
theValue = node;
} else if (node.isObject()) {
// this is an object so just send it back so that it can be
// reprocessed to its primitive components
theValue = node;
} else {
theValue = node.textValue();
}
}
return theValue;
}
}
static class ParseEdge extends ParseElement {
private static final String MESSAGE_EDGE = "edge";
private static final String TYPE_NAME_NODE_NAME = Constants.VERTEX_TYPE_PROPERTY_KEY;
@Override
......@@ -118,7 +86,7 @@ public class JsonNodeParsers {
@Override
Object getId(JsonNode node) {
return getTypedValueFromJsonNode(node.get(GraphSONTokensTP2._ID));
return utility.getTypedValueFromJsonNode(node.get(GraphSONTokensTP2._ID));
}
@Override
......@@ -163,7 +131,7 @@ public class JsonNodeParsers {
@Override
Object getId(JsonNode node) {
return getTypedValueFromJsonNode(node.get(GraphSONTokensTP2._ID));
return utility.getTypedValueFromJsonNode(node.get(GraphSONTokensTP2._ID));
}
@Override
......
......@@ -108,7 +108,7 @@ public class JsonNodeProcessManager {
display("commit-size: {}: Done!", size);
}
private void updateSchema(Map<String, Object> schema, org.apache.tinkerpop.shaded.jackson.databind.JsonNode node) {
private void updateSchema(Map<String, Object> schema, JsonNode node) {
synchronized (graph) {
String typeName = parseElement.getType(node);
......@@ -142,7 +142,7 @@ public class JsonNodeProcessManager {
try {
Thread.sleep(WAIT_DURATION_AFTER_COMMIT_EXCEPTION);
for (org.apache.tinkerpop.shaded.jackson.databind.JsonNode n : nodes) {
for (JsonNode n : nodes) {
parseElement.parse(bulkLoadGraph, cache, n);
}
commitBulk();
......
......@@ -18,6 +18,7 @@
package org.apache.atlas.repository.graphdb.janus.migration;
import com.google.common.annotations.VisibleForTesting;
import org.apache.atlas.utils.LruCache;
import org.apache.tinkerpop.gremlin.structure.Edge;
import org.apache.tinkerpop.gremlin.structure.Graph;
......@@ -33,8 +34,11 @@ import static org.apache.atlas.repository.Constants.VERTEX_ID_IN_IMPORT_KEY;
public class MappedElementCache {
private static final Logger LOG = LoggerFactory.getLogger(MappedElementCache.class);
private final Map<Object, Vertex> lruVertexCache = new LruCache<>(500, 100000);
private final Map<String, String> lruEdgeCache = new LruCache<>(500, 100000);
@VisibleForTesting
final Map<Object, Vertex> lruVertexCache = new LruCache<>(500, 100000);
@VisibleForTesting
final Map<String, String> lruEdgeCache = new LruCache<>(500, 100000);
public Vertex getMappedVertex(Graph gr, Object key) {
try {
......@@ -83,7 +87,8 @@ public class MappedElementCache {
}
}
private Vertex fetchVertex(Graph gr, Object key) {
@VisibleForTesting
Vertex fetchVertex(Graph gr, Object key) {
try {
return gr.traversal().V().has(VERTEX_ID_IN_IMPORT_KEY, key).next();
} catch (Exception ex) {
......@@ -92,7 +97,8 @@ public class MappedElementCache {
}
}
private Edge fetchEdge(Graph gr, String key) {
@VisibleForTesting
Edge fetchEdge(Graph gr, String key) {
try {
return gr.traversal().E().has(EDGE_ID_IN_IMPORT_KEY, key).next();
} catch (Exception ex) {
......@@ -101,16 +107,8 @@ public class MappedElementCache {
}
}
public void clearVertexCache() {
public void clearAll() {
lruVertexCache.clear();
}
public void clearEdgeCache() {
lruEdgeCache.clear();
}
public void clearAll() {
clearVertexCache();
clearEdgeCache();
}
}
......@@ -29,7 +29,7 @@ import org.slf4j.LoggerFactory;
import java.util.concurrent.BlockingQueue;
public class PostProcessManager {
private static class Consumer extends WorkItemConsumer<Object> {
static class Consumer extends WorkItemConsumer<Object> {
private static final Logger LOG = LoggerFactory.getLogger(Consumer.class);
private final Graph bulkLoadGraph;
......
......@@ -18,6 +18,8 @@
package org.apache.atlas.repository.graphdb.janus.migration;
import com.google.common.annotations.VisibleForTesting;
import org.apache.atlas.model.impexp.MigrationStatus;
import org.apache.atlas.repository.Constants;
import org.apache.tinkerpop.gremlin.process.traversal.dsl.graph.GraphTraversal;
import org.apache.tinkerpop.gremlin.process.traversal.dsl.graph.GraphTraversalSource;
......@@ -43,7 +45,8 @@ public class ReaderStatusManager {
public static final String STATUS_SUCCESS = "SUCCESS";
public static final String STATUS_FAILED = "FAILED";
private Object migrationStatusId = null;
@VisibleForTesting
Object migrationStatusId = null;
private Vertex migrationStatus = null;
public ReaderStatusManager(Graph graph, Graph bulkLoadGraph) {
......@@ -71,8 +74,11 @@ public class ReaderStatusManager {
public void update(Graph graph, Long counter) {
migrationStatus.property(CURRENT_INDEX_PROPERTY, counter);
if(graph.features().graph().supportsTransactions()) {
graph.tx().commit();
}
}
public void update(Graph graph, Long counter, String status) {
migrationStatus.property(OPERATION_STATUS_PROPERTY, status);
......@@ -91,7 +97,7 @@ public class ReaderStatusManager {
return g.V(migrationStatusId).next();
}
private Vertex fetchUsingTypeName(GraphTraversalSource g) {
private static Vertex fetchUsingTypeName(GraphTraversalSource g) {
GraphTraversal src = g.V().has(Constants.ENTITY_TYPE_PROPERTY_KEY, MIGRATION_STATUS_TYPE_NAME);
return src.hasNext() ? (Vertex) src.next() : null;
}
......@@ -109,8 +115,42 @@ public class ReaderStatusManager {
migrationStatusId = v.id();
if(rGraph.features().graph().supportsTransactions()) {
rGraph.tx().commit();
}
LOG.info("migrationStatus vertex created! v[{}]", migrationStatusId);
}
public static MigrationStatus updateFromVertex(Graph graph, MigrationStatus ms) {
Vertex vertex = fetchUsingTypeName(graph.traversal());
if(ms == null) {
ms = new MigrationStatus();
}
ms.setStartTime((Date) vertex.property(START_TIME_PROPERTY).value());
ms.setEndTime((Date) vertex.property(END_TIME_PROPERTY).value());
ms.setCurrentIndex((Long) vertex.property(CURRENT_INDEX_PROPERTY).value());
ms.setOperationStatus((String) vertex.property(OPERATION_STATUS_PROPERTY).value());
ms.setTotalCount((Long) vertex.property(TOTAL_COUNT_PROPERTY).value());
return ms;
}
public static MigrationStatus get(Graph graph) {
MigrationStatus ms = new MigrationStatus();
try {
Vertex v = fetchUsingTypeName(graph.traversal());
ms.setStartTime((Date) v.property(START_TIME_PROPERTY).value());
ms.setEndTime((Date) v.property(END_TIME_PROPERTY).value());
ms.setCurrentIndex((long) v.property(CURRENT_INDEX_PROPERTY).value());
ms.setOperationStatus((String) v.property(OPERATION_STATUS_PROPERTY).value());
ms.setTotalCount((long) v.property(TOTAL_COUNT_PROPERTY).value());
} catch (Exception ex) {
LOG.error("get: failed!", ex);
}
return ms;
}
}
......@@ -18,14 +18,9 @@
package org.apache.atlas.repository.graphdb.janus.migration;
import org.apache.commons.lang.StringUtils;
import org.apache.tinkerpop.gremlin.structure.Vertex;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.util.ArrayList;
import java.util.Collection;
import java.util.List;
import java.util.Map;
public class RelationshipTypeCache {
......
......@@ -86,7 +86,7 @@ public abstract class WorkItemConsumer<T> implements Runnable {
protected abstract void processItem(T item);
private void updateCommitTime(long commitTime) {
protected void updateCommitTime(long commitTime) {
if (this.maxCommitTimeSeconds < commitTime) {
this.maxCommitTimeSeconds = commitTime;
}
......
......@@ -55,7 +55,7 @@ public class WorkItemManager<T, U extends WorkItemConsumer> {
public void shutdown() throws InterruptedException {
int avgCommitTimeSeconds = getAvgCommitTimeSeconds() * 2;
LOG.info("WorkItemManager: Shutdown started. Will wait for: {} seconds...", avgCommitTimeSeconds);
LOG.info("WorkItemManager: Shutdown started. Will wait for: {} minutes...", avgCommitTimeSeconds);
service.shutdown();
service.awaitTermination(avgCommitTimeSeconds, TimeUnit.MINUTES);
......
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS 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.repository.graphdb.janus.migration;
import org.apache.commons.io.FileUtils;
import org.apache.tinkerpop.gremlin.process.traversal.dsl.graph.GraphTraversal;
import org.apache.tinkerpop.gremlin.structure.Vertex;
import org.apache.tinkerpop.gremlin.structure.io.graphson.GraphSONMapper;
import org.apache.tinkerpop.gremlin.tinkergraph.structure.TinkerGraph;
import org.apache.tinkerpop.shaded.jackson.databind.JsonNode;
import org.apache.tinkerpop.shaded.jackson.databind.ObjectMapper;
import org.testng.ITestContext;
import org.testng.annotations.BeforeClass;
import org.testng.annotations.DataProvider;
import java.io.File;
import java.io.IOException;
import java.nio.file.Paths;
import java.util.HashMap;
import static org.testng.AssertJUnit.assertTrue;
public class BaseUtils {
private static final String resourcesDirRelativePath = "/src/test/resources/";
private String resourceDir;
protected final RelationshipTypeCache emptyRelationshipCache = new RelationshipTypeCache(new HashMap<>());
protected GraphSONUtility graphSONUtility;
protected Object[][] getJsonNodeFromFile(String s) throws IOException {
File f = new File(getFilePath(s));
return new Object[][]{{getEntityNode(FileUtils.readFileToString(f))}};
}
protected String getFilePath(String fileName) {
return Paths.get(resourceDir, fileName).toString();
}
@BeforeClass
public void setup() {
resourceDir = System.getProperty("user.dir") + resourcesDirRelativePath;
graphSONUtility = new GraphSONUtility(emptyRelationshipCache);
}
protected Object getId(JsonNode node) {
GraphSONUtility gu = graphSONUtility;
return gu.getTypedValueFromJsonNode(node.get(GraphSONTokensTP2._ID));
}
private JsonNode getEntityNode(String json) throws IOException {
GraphSONMapper.Builder builder = GraphSONMapper.build();
final ObjectMapper mapper = builder.embedTypes(false).create().createMapper();
return mapper.readTree(json);
}
protected void addVertex(TinkerGraph tg, JsonNode node) {
GraphSONUtility utility = new GraphSONUtility(emptyRelationshipCache);
utility.vertexFromJson(tg, node);
}
protected void addEdge(TinkerGraph tg, MappedElementCache cache) throws IOException {
GraphSONUtility gu = graphSONUtility;
gu.vertexFromJson(tg, (JsonNode) (getDBV(null)[0][0]));
gu.vertexFromJson(tg, (JsonNode) (getTableV(null))[0][0]);
gu.edgeFromJson(tg, cache, (JsonNode) getEdge(null)[0][0]);
}
protected Vertex fetchTableVertex(TinkerGraph tg) {
GraphTraversal query = tg.traversal().V().has("__typeName", "hive_table");
assertTrue(query.hasNext());
return (Vertex) query.next();
}
@DataProvider(name = "col1")
public Object[][] getCol1(ITestContext context) throws IOException {
return getJsonNodeFromFile("col-legacy.json");
}
@DataProvider(name = "dbType")
public Object[][] getDbType(ITestContext context) throws IOException {
return getJsonNodeFromFile("db-type-legacy.json");
}
@DataProvider(name = "edge")
public Object[][] getEdge(ITestContext context) throws IOException {
return getJsonNodeFromFile("edge-legacy.json");
}
@DataProvider(name = "dbV")
public Object[][] getDBV(ITestContext context) throws IOException {
return getJsonNodeFromFile("db-v-65544.json");
}
@DataProvider(name = "tableV")
public Object[][] getTableV(ITestContext context) throws IOException {
return getJsonNodeFromFile("table-v-147504.json");
}
}
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.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.repository.graphdb.janus.migration;
import org.apache.tinkerpop.gremlin.process.traversal.dsl.graph.GraphTraversal;
import org.apache.tinkerpop.gremlin.structure.Vertex;
import org.apache.tinkerpop.gremlin.tinkergraph.structure.TinkerGraph;
import org.eclipse.jetty.util.BlockingArrayQueue;
import org.testng.annotations.Test;
import java.io.IOException;
import java.util.List;
import java.util.concurrent.BlockingQueue;
import static org.testng.Assert.assertEquals;
import static org.testng.Assert.assertNotNull;
import static org.testng.AssertJUnit.assertTrue;
public class GraphSONUtilityPostProcessTest extends BaseUtils {
final String HIVE_COLUMNS_PROPERTY = "hive_table.columns";
final String edgeId1 = "816u-35tc-ao0l-47so";
final String edgeId2 = "82rq-35tc-ao0l-2glc";
final String edgeId1x = "816u-35tc-ao0l-xxxx";
final String edgeId2x = "82rq-35tc-ao0l-xxxx";
private TinkerGraph tg;
private MappedElementCache cache = new MappedElementCache();
private Vertex tableV;
@Test
public void noRefNoUpdate() throws IOException {
tg = TinkerGraph.open();
graphSONUtility = new GraphSONUtility(emptyRelationshipCache);
addEdge(tg, cache);
tableV = fetchTableVertex(tg);
assertNotNull(tableV);
assertListProperty(HIVE_COLUMNS_PROPERTY, edgeId1, edgeId2, tableV);
graphSONUtility.replaceReferencedEdgeIdForList(tg, cache, tableV, HIVE_COLUMNS_PROPERTY);
assertListProperty(HIVE_COLUMNS_PROPERTY, edgeId1, edgeId2, tableV);
}
@Test(dependsOnMethods = "noRefNoUpdate")
public void refFoundVertexUpdated() throws IOException {
cache.lruEdgeCache.put(edgeId1, edgeId1x);
cache.lruEdgeCache.put(edgeId2, edgeId2x);
graphSONUtility.replaceReferencedEdgeIdForList(tg, cache, tableV, HIVE_COLUMNS_PROPERTY);
assertListProperty(HIVE_COLUMNS_PROPERTY, edgeId1x, edgeId2x, tableV);
}
@Test(dependsOnMethods = "refFoundVertexUpdated")
public void updateUsingPostProcessConsumer() throws IOException {
MappedElementCache cache = new MappedElementCache();
BlockingQueue<Object> bc = new BlockingArrayQueue<>();
PostProcessManager.Consumer consumer = new PostProcessManager.Consumer(bc, tg, graphSONUtility,
new String[] {HIVE_COLUMNS_PROPERTY}, cache, 5);
cache.lruEdgeCache.put(edgeId1x, edgeId1);
cache.lruEdgeCache.put(edgeId2x, edgeId2);
consumer.processItem(tableV.id());
assertListProperty(HIVE_COLUMNS_PROPERTY, edgeId1, edgeId2, tableV);
}
private void assertListProperty(String HIVE_COLUMNS_PROPERTY, String edgeId1, String edgeId2, Vertex tableV) {
assertTrue(tableV.property(HIVE_COLUMNS_PROPERTY).isPresent());
List list = (List) tableV.property(HIVE_COLUMNS_PROPERTY).value();
assertEquals(list.size(), 2);
assertEquals(list.get(0), edgeId1);
assertEquals(list.get(1), edgeId2);
}
}
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS 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.repository.graphdb.janus.migration;
import org.apache.tinkerpop.gremlin.structure.Edge;
import org.apache.tinkerpop.gremlin.structure.Vertex;
import org.apache.tinkerpop.gremlin.tinkergraph.structure.TinkerGraph;
import org.apache.tinkerpop.shaded.jackson.databind.JsonNode;
import org.testng.Assert;
import org.testng.ITestContext;
import org.testng.annotations.DataProvider;
import org.testng.annotations.Test;
import java.io.IOException;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.Map;
import static org.apache.atlas.repository.Constants.EDGE_ID_IN_IMPORT_KEY;
import static org.apache.atlas.repository.Constants.VERTEX_ID_IN_IMPORT_KEY;
import static org.testng.Assert.*;
public class GraphSONUtilityTest extends BaseUtils {
@Test(dataProvider = "col1")
public void idFetch(JsonNode node) {
Object o = GraphSONUtility.getTypedValueFromJsonNode(node.get(GraphSONTokensTP2._ID));
assertNotNull(o);
assertEquals((int) o, 98336);
}
@Test(dataProvider = "col1")
public void verifyReadProperties(JsonNode node) {
Map<String, Object> props = GraphSONUtility.readProperties(node);
assertEquals(props.get("__superTypeNames").getClass(), ArrayList.class);
assertEquals(props.get("Asset.name").getClass(), String.class);
assertEquals(props.get("hive_column.position").getClass(), Integer.class);
assertEquals(props.get("__timestamp").getClass(), Long.class);
assertNotNull(props);
}
@Test(dataProvider = "col1")
public void dataNodeReadAndVertexAddedToGraph(JsonNode entityNode) throws IOException {
TinkerGraph tg = TinkerGraph.open();
GraphSONUtility gu = new GraphSONUtility(emptyRelationshipCache);
Map<String, Object> map = gu.vertexFromJson(tg, entityNode);
assertNull(map);
assertEquals((long) tg.traversal().V().count().next(), 1L);
Vertex v = tg.vertices().next();
assertTrue(v.property(VERTEX_ID_IN_IMPORT_KEY).isPresent());
}
@Test(dataProvider = "dbType")
public void typeNodeReadAndVertexNotAddedToGraph(JsonNode entityNode) throws IOException {
TinkerGraph tg = TinkerGraph.open();
GraphSONUtility gu = new GraphSONUtility(emptyRelationshipCache);
gu.vertexFromJson(tg, entityNode);
Assert.assertEquals((long) tg.traversal().V().count().next(), 0L);
}
@Test
public void edgeReadAndAddedToGraph() throws IOException {
TinkerGraph tg = TinkerGraph.open();
GraphSONUtility gu = new GraphSONUtility(emptyRelationshipCache);
Map<String, Object> m = null;
m = gu.vertexFromJson(tg, (JsonNode) (getDBV(null)[0][0]));
assertNull(m);
m = gu.vertexFromJson(tg, (JsonNode) (getTableV(null))[0][0]);
assertNull(m);
m = gu.edgeFromJson(tg, new MappedElementCache(), (JsonNode) getEdge(null)[0][0]);
assertNull(m);
Assert.assertEquals((long) tg.traversal().V().count().next(), 2L);
Assert.assertEquals((long) tg.traversal().E().count().next(), 1L);
Edge e = tg.edges().next();
assertTrue(e.property(EDGE_ID_IN_IMPORT_KEY).isPresent());
}
}
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS 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.repository.graphdb.janus.migration;
import org.apache.tinkerpop.gremlin.structure.Edge;
import org.apache.tinkerpop.gremlin.structure.Element;
import org.apache.tinkerpop.gremlin.structure.Vertex;
import org.apache.tinkerpop.gremlin.tinkergraph.structure.TinkerGraph;
import org.apache.tinkerpop.shaded.jackson.databind.JsonNode;
import org.testng.annotations.Test;
import java.io.IOException;
import java.util.HashMap;
import java.util.Map;
import static org.testng.Assert.assertFalse;
import static org.testng.Assert.assertTrue;
import static org.testng.AssertJUnit.assertEquals;
import static org.testng.AssertJUnit.assertNotNull;
public class JsonNodeParsersTest extends BaseUtils {
@Test(dataProvider = "col1")
public void parseVertex(JsonNode nd) {
final int COL1_ORIGINAL_ID = 98336;
Object nodeId = getId(nd);
TinkerGraph tg = TinkerGraph.open();
JsonNodeParsers.ParseElement pe = new JsonNodeParsers.ParseVertex();
pe.setContext(graphSONUtility);
pe.parse(tg, new MappedElementCache(), nd);
Vertex v = tg.vertices().next();
Vertex vUsingPe = (Vertex) pe.get(tg, nodeId);
Vertex vUsingOriginalId = (Vertex) pe.getByOriginalId(tg, COL1_ORIGINAL_ID);
Vertex vUsingOriginalId2 = (Vertex) pe.getByOriginalId(tg, nd);
updateParseElement(tg, pe, vUsingPe);
assertNotNull(v);
assertNotNull(vUsingPe);
assertNotNull(vUsingOriginalId);
assertEquals(v.id(), vUsingPe.id());
assertEquals(nodeId, pe.getId(nd));
assertFalse(pe.isTypeNode(nd));
assertEquals(pe.getType(nd), "\"hive_column\"");
assertEquals(vUsingOriginalId.id(), v.id());
assertEquals(vUsingOriginalId2.id(), v.id());
assertProperties(vUsingPe);
}
@Test(dataProvider = "edge")
public void parseEdge(JsonNode nd) throws IOException {
final String EDGE_ORIGINAL_ID = "8k5i-35tc-acyd-1eko";
Object nodeId = getId(nd);
TinkerGraph tg = TinkerGraph.open();
MappedElementCache cache = new MappedElementCache();
JsonNodeParsers.ParseElement peVertex = new JsonNodeParsers.ParseVertex();
peVertex.setContext(graphSONUtility);
peVertex.parse(tg, cache, (JsonNode) (getDBV(null)[0][0]));
peVertex.parse(tg, cache, (JsonNode) (getTableV(null)[0][0]));
JsonNodeParsers.ParseElement pe = new JsonNodeParsers.ParseEdge();
pe.setContext(graphSONUtility);
pe.parse(tg, cache, (JsonNode) getEdge(null)[0][0]);
updateParseElement(tg, pe, nodeId);
Edge e = tg.edges().next();
Edge eUsingPe = (Edge) pe.get(tg, nodeId);
Edge eUsingOriginalId = (Edge) pe.getByOriginalId(tg, EDGE_ORIGINAL_ID);
Edge eUsingOriginalId2 = (Edge) pe.getByOriginalId(tg, nd);
assertNotNull(e);
assertNotNull(eUsingPe);
assertNotNull(eUsingOriginalId);
assertEquals(e.id(), eUsingPe.id());
assertEquals(nodeId, pe.getId(nd));
assertFalse(pe.isTypeNode(nd));
assertEquals(eUsingOriginalId.id(), e.id());
assertEquals(eUsingOriginalId2.id(), e.id());
assertProperties(e);
}
private void updateParseElement(TinkerGraph tg, JsonNodeParsers.ParseElement pe, Object nodeId) {
Map<String, Object> props = new HashMap<>();
props.put("k1", "v1");
props.put("k2", "v2");
pe.update(tg, nodeId, props);
}
private void assertProperties(Element v) {
assertNotNull(v);
assertTrue(v.property("k1").isPresent());
assertTrue(v.property("k2").isPresent());
assertEquals(v.property("k1").value(), "v1");
}
}
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS 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.repository.graphdb.janus.migration;
import org.apache.tinkerpop.gremlin.structure.Vertex;
import org.apache.tinkerpop.gremlin.tinkergraph.structure.TinkerGraph;
import org.apache.tinkerpop.shaded.jackson.databind.JsonNode;
import org.testng.ITestContext;
import org.testng.annotations.DataProvider;
import org.testng.annotations.Test;
import java.io.IOException;
import static org.testng.Assert.*;
public class MappedElementCacheTest extends BaseUtils {
@Test(dataProvider = "col1")
public void vertexFetch(JsonNode node) {
MappedElementCache cache = new MappedElementCache();
TinkerGraph tg = TinkerGraph.open();
addVertex(tg, node);
Vertex vx = cache.getMappedVertex(tg, 98336);
assertNotNull(vx);
assertEquals(cache.lruVertexCache.size(), 1);
assertEquals(cache.lruEdgeCache.size(), 0);
}
@Test
public void edgeFetch() throws IOException {
MappedElementCache cache = new MappedElementCache();
TinkerGraph tg = TinkerGraph.open();
addEdge(tg, cache);
assertEquals(cache.lruVertexCache.size(), 2);
assertEquals(cache.lruEdgeCache.size(), 0);
}
@Test
public void nonExistentVertexReturnsNull() {
TinkerGraph tg = TinkerGraph.open();
MappedElementCache cache = new MappedElementCache();
assertNull(cache.fetchVertex(tg, 1111));
assertNull(cache.fetchEdge(tg, "abcd"));
}
@DataProvider(name = "col1")
public Object[][] getCol1(ITestContext context) throws IOException {
return getJsonNodeFromFile("col-legacy.json");
}
}
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS 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.repository.graphdb.janus.migration;
import org.apache.atlas.model.impexp.MigrationStatus;
import org.apache.tinkerpop.gremlin.tinkergraph.structure.TinkerGraph;
import org.testng.annotations.Test;
import static org.testng.Assert.assertEquals;
import static org.testng.Assert.assertNotNull;
public class ReaderStatusManagerTest {
@Test
public void createsNewStatusNode() {
TinkerGraph tg = TinkerGraph.open();
ReaderStatusManager sm = new ReaderStatusManager(tg, tg);
assertEquals(sm.getStartIndex(), 0L);
assertNotNull(tg.traversal().V(sm.migrationStatusId).next());
MigrationStatus ms = ReaderStatusManager.updateFromVertex(tg, null);
assertEquals(ms.getCurrentIndex(), 0L);
assertEquals(ms.getTotalCount(), 0L);
assertEquals(ms.getOperationStatus(), "NOT STARTED");
assertNotNull(ms.getStartTime());
assertNotNull(ms.getEndTime());
}
@Test
public void verifyUpdates() {
long expectedTotalCount = 1001L;
String expectedOperationStatus = "SUCCESS";
TinkerGraph tg = TinkerGraph.open();
ReaderStatusManager sm = new ReaderStatusManager(tg, tg);
sm.update(tg, 1000L, "IN PROGRESS");
sm.end(tg, expectedTotalCount, expectedOperationStatus);
MigrationStatus ms = ReaderStatusManager.updateFromVertex(tg, null);
assertEquals(ms.getCurrentIndex(), expectedTotalCount);
assertEquals(ms.getTotalCount(), expectedTotalCount);
assertEquals(ms.getOperationStatus(), expectedOperationStatus);
}
}
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS 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.repository.graphdb.janus.migration;
import org.apache.atlas.repository.graphdb.janus.migration.pc.WorkItemConsumer;
import org.testng.annotations.Test;
import java.util.concurrent.BlockingQueue;
import java.util.concurrent.LinkedBlockingQueue;
import static org.testng.Assert.assertFalse;
import static org.testng.Assert.assertTrue;
public class WorkItemConsumerTest {
private class IntegerConsumerSpy extends WorkItemConsumer<Integer> {
boolean commitDirtyCalled = false;
private boolean updateCommitTimeCalled;
public IntegerConsumerSpy(BlockingQueue<Integer> queue) {
super(queue);
}
@Override
protected void doCommit() {
}
@Override
protected void processItem(Integer item) {
}
@Override
protected void commitDirty() {
commitDirtyCalled = true;
super.commitDirty();
}
@Override
protected void updateCommitTime(long commitTime) {
updateCommitTimeCalled = true;
}
public boolean isCommitDirtyCalled() {
return commitDirtyCalled;
}
public boolean isUpdateCommitTimeCalled() {
return updateCommitTimeCalled;
}
}
@Test
public void callingRunOnEmptyQueueCallsDoesNotCallCommitDirty() {
BlockingQueue<Integer> bc = new LinkedBlockingQueue<>(5);
IntegerConsumerSpy ic = new IntegerConsumerSpy(bc);
ic.run();
assertTrue(bc.isEmpty());
assertTrue(ic.isCommitDirtyCalled());
assertFalse(ic.isUpdateCommitTimeCalled());
}
@Test
public void runOnQueueRemovesItemFromQueuCallsCommitDirty() {
BlockingQueue<Integer> bc = new LinkedBlockingQueue<>(5);
bc.add(1);
IntegerConsumerSpy ic = new IntegerConsumerSpy(bc);
ic.run();
assertTrue(bc.isEmpty());
assertTrue(ic.isCommitDirtyCalled());
assertTrue(ic.isUpdateCommitTimeCalled());
}
}
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.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.repository.graphdb.janus.migration;
import org.apache.atlas.repository.graphdb.janus.migration.pc.WorkItemBuilder;
import org.apache.atlas.repository.graphdb.janus.migration.pc.WorkItemConsumer;
import org.apache.atlas.repository.graphdb.janus.migration.pc.WorkItemManager;
import org.apache.commons.lang3.RandomUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.testng.SkipException;
import org.testng.annotations.Test;
import java.util.concurrent.BlockingQueue;
import java.util.concurrent.ConcurrentLinkedQueue;
import static org.testng.Assert.assertEquals;
public class WorkItemManagerTest {
private static final Logger LOG = LoggerFactory.getLogger(WorkItemManagerTest.class);
private class IntegerConsumer extends WorkItemConsumer<Integer> {
private final ConcurrentLinkedQueue<Integer> target;
public IntegerConsumer(BlockingQueue<Integer> queue, ConcurrentLinkedQueue<Integer> target) {
super(queue);
this.target = target;
}
@Override
protected void doCommit() {
try {
Thread.sleep(20 * RandomUtils.nextInt(10, 15));
} catch (InterruptedException e) {
e.printStackTrace();
}
}
@Override
protected void processItem(Integer item) {
LOG.info("adding: {}: size: {}", item, target.size());
target.add(item);
}
}
private class IntegerConsumerBuilder implements WorkItemBuilder<IntegerConsumer, Integer> {
ConcurrentLinkedQueue<Integer> integers = new ConcurrentLinkedQueue<>();
@Override
public IntegerConsumer build(BlockingQueue<Integer> queue) {
return new IntegerConsumer(queue, integers);
}
}
@Test
public void oneWorkerSequences() {
IntegerConsumerBuilder cb = new IntegerConsumerBuilder();
int numberOfItems = 10;
try {
WorkItemManager<Integer, WorkItemConsumer> wi = getWorkItemManger(cb, 1);
for (int i = 0; i < numberOfItems; i++) {
wi.produce(i);
}
wi.shutdown();
} catch (InterruptedException e) {
throw new SkipException("Test skipped!");
}
assertEquals(cb.integers.size(), numberOfItems);
Integer[] ints = cb.integers.toArray(new Integer[]{});
for (int i = 0; i < numberOfItems; i++) {
assertEquals(ints[i], i, i);
}
}
@Test
public void multipleWorkersUnpredictableSequence() {
IntegerConsumerBuilder cb = new IntegerConsumerBuilder();
int numberOfItems = 100;
try {
WorkItemManager<Integer, WorkItemConsumer> wi = getWorkItemManger(cb, 5);
for (int i = 0; i < numberOfItems; i++) {
wi.produce(i);
}
wi.shutdown();
} catch (InterruptedException e) {
throw new SkipException("Test skipped!");
}
assertEquals(cb.integers.size(), numberOfItems);
}
private WorkItemManager<Integer, WorkItemConsumer> getWorkItemManger(IntegerConsumerBuilder cb, int numWorkers) {
return new WorkItemManager<>(cb, 5, numWorkers);
}
}
This source diff could not be displayed because it is too large. You can view the blob instead.
{
"Asset.name": {
"type": "string",
"value": "col4"
},
"hive_column.type": {
"type": "string",
"value": "string"
},
"__modifiedBy": {
"type": "string",
"value": "anonymous"
},
"__state": {
"type": "string",
"value": "ACTIVE"
},
"entityText": {
"type": "string",
"value": "hive_column owner anonymous qualifiedName stocks.test_table.col4@cl1 name col4 position 0 type string table "
},
"Referenceable.qualifiedName": {
"type": "string",
"value": "stocks.test_table.col4@cl1"
},
"__guid": {
"type": "string",
"value": "0693682a-30ae-4fec-a533-179e572792ce"
},
"__version": {
"type": "integer",
"value": 0
},
"__superTypeNames": {
"type": "list",
"value": [{
"type": "string",
"value": "Asset"
}, {
"type": "string",
"value": "DataSet"
}, {
"type": "string",
"value": "Referenceable"
}]
},
"__createdBy": {
"type": "string",
"value": "anonymous"
},
"__typeName": {
"type": "string",
"value": "hive_column"
},
"__modificationTimestamp": {
"type": "long",
"value": 1522693838471
},
"Asset.owner": {
"type": "string",
"value": "anonymous"
},
"hive_column.position": {
"type": "integer",
"value": 0
},
"__timestamp": {
"type": "long",
"value": 1522693826849
},
"_id": 98336,
"_type": "vertex"
}
{
"__type.name": {
"type": "string",
"value": "hive_db"
},
"__type.hive_db.parameters": {
"type": "string",
"value": "{\"multiplicity\":\"{\\\"lower\\\":0,\\\"upper\\\":1,\\\"isUnique\\\":false}\",\"isIndexable\":false,\"isComposite\":false,\"reverseAttributeName\":null,\"dataType\":\"map<string,string>\",\"name\":\"parameters\",\"isUnique\":false}"
},
"__modifiedBy": {
"type": "string",
"value": "root"
},
"__type.hive_db": {
"type": "list",
"value": [{
"type": "string",
"value": "clusterName"
}, {
"type": "string",
"value": "location"
}, {
"type": "string",
"value": "parameters"
}, {
"type": "string",
"value": "ownerType"
}]
},
"__type.options": {
"type": "string",
"value": "null"
},
"__guid": {
"type": "string",
"value": "b2685ea8-16c5-4d54-88f2-41b1d66bd1fb"
},
"__version": {
"type": "long",
"value": 1
},
"__type.hive_db.clusterName": {
"type": "string",
"value": "{\"multiplicity\":\"{\\\"lower\\\":1,\\\"upper\\\":1,\\\"isUnique\\\":false}\",\"isIndexable\":true,\"isComposite\":false,\"reverseAttributeName\":null,\"dataType\":\"string\",\"name\":\"clusterName\",\"isUnique\":false}"
},
"__type.category": {
"type": "string",
"value": "CLASS"
},
"__type.version": {
"type": "string",
"value": "1.0"
},
"__type.hive_db.location": {
"type": "string",
"value": "{\"multiplicity\":\"{\\\"lower\\\":0,\\\"upper\\\":1,\\\"isUnique\\\":false}\",\"isIndexable\":false,\"isComposite\":false,\"reverseAttributeName\":null,\"dataType\":\"string\",\"name\":\"location\",\"isUnique\":false}"
},
"__createdBy": {
"type": "string",
"value": "root"
},
"__modificationTimestamp": {
"type": "long",
"value": 1522693758158
},
"__type": {
"type": "string",
"value": "typeSystem"
},
"__type.description": {
"type": "string",
"value": "hive_db"
},
"__timestamp": {
"type": "long",
"value": 1522693758158
},
"__type.hive_db.ownerType": {
"type": "string",
"value": "{\"multiplicity\":\"{\\\"lower\\\":0,\\\"upper\\\":1,\\\"isUnique\\\":false}\",\"isIndexable\":false,\"isComposite\":false,\"reverseAttributeName\":null,\"dataType\":\"hive_principal_type\",\"name\":\"ownerType\",\"isUnique\":false}"
},
"_id": 16392,
"_type": "vertex"
}
{
"hive_db.parameters": {
"type": "list",
"value": []
},
"Asset.name": {
"type": "string",
"value": "stocks"
},
"hive_db.ownerType": {
"type": "string",
"value": "USER"
},
"__modifiedBy": {
"type": "string",
"value": "anonymous"
},
"__state": {
"type": "string",
"value": "ACTIVE"
},
"entityText": {
"type": "string",
"value": "hive_db owner anonymous ownerType USER qualifiedName stocks@cl1 clusterName cl1 name stocks location hdfs://localhost.localdomain:8020/apps/hive/warehouse/stocks.db "
},
"Referenceable.qualifiedName": {
"type": "string",
"value": "stocks@cl1"
},
"__guid": {
"type": "string",
"value": "229b7fd4-e46e-4338-9e44-18ce630eb5bf"
},
"__version": {
"type": "integer",
"value": 0
},
"__superTypeNames": {
"type": "list",
"value": [{
"type": "string",
"value": "Asset"
}, {
"type": "string",
"value": "Referenceable"
}]
},
"__createdBy": {
"type": "string",
"value": "anonymous"
},
"__typeName": {
"type": "string",
"value": "hive_db"
},
"__modificationTimestamp": {
"type": "long",
"value": 1522693838471
},
"hive_db.clusterName": {
"type": "string",
"value": "cl1"
},
"Asset.owner": {
"type": "string",
"value": "anonymous"
},
"hive_db.location": {
"type": "string",
"value": "hdfs://localhost.localdomain:8020/apps/hive/warehouse/stocks.db"
},
"__timestamp": {
"type": "long",
"value": 1522693806944
},
"_id": 65544,
"_type": "vertex"
}
\ No newline at end of file
{
"__modifiedBy": {
"type": "string",
"value": "anonymous"
},
"__state": {
"type": "string",
"value": "ACTIVE"
},
"__createdBy": {
"type": "string",
"value": "anonymous"
},
"__modificationTimestamp": {
"type": "long",
"value": 1522693835017
},
"__timestamp": {
"type": "long",
"value": 1522693835017
},
"_id": "8k5i-35tc-acyd-1eko",
"_type": "edge",
"_outV": 147504,
"_inV": 65544,
"_label": "__hive_table.db"
}
{
"hive_table.createTime": {
"type": "long",
"value": 1522693834000
},
"hive_table.tableType": {
"type": "string",
"value": "VIRTUAL_VIEW"
},
"Asset.name": {
"type": "string",
"value": "test_table_view"
},
"__modifiedBy": {
"type": "string",
"value": "anonymous"
},
"__state": {
"type": "string",
"value": "ACTIVE"
},
"entityText": {
"type": "string",
"value": "hive_table owner anonymous temporary false lastAccessTime Mon Apr 02 11:30:34 PDT 2018 qualifiedName stocks.test_table_view@cl1 columns viewExpandedText select `test_table`.`col1`, `test_table`.`col2`, `test_table`.`col4` from `stocks`.`test_table` sd tableType VIRTUAL_VIEW createTime Mon Apr 02 11:30:34 PDT 2018 name test_table_view partitionKeys parameters transient_lastDdlTime 1522693834 db retention 0 viewOriginalText select col1, col2, col4 from test_table "
},
"Referenceable.qualifiedName": {
"type": "string",
"value": "stocks.test_table_view@cl1"
},
"hive_table.parameters.transient_lastDdlTime": {
"type": "string",
"value": "1522693834"
},
"hive_table.parameters": {
"type": "list",
"value": [{
"type": "string",
"value": "transient_lastDdlTime"
}]
},
"hive_table.retention": {
"type": "integer",
"value": 0
},
"hive_table.partitionKeys": {
"type": "list",
"value": [{
"type": "string",
"value": "8dty-35tc-amfp-23xs"
}]
},
"__guid": {
"type": "string",
"value": "111091f1-2661-4946-b09b-64e28f10c109"
},
"hive_table.temporary": {
"type": "boolean",
"value": false
},
"__version": {
"type": "integer",
"value": 0
},
"__superTypeNames": {
"type": "list",
"value": [{
"type": "string",
"value": "Asset"
}, {
"type": "string",
"value": "DataSet"
}, {
"type": "string",
"value": "Referenceable"
}]
},
"hive_table.viewExpandedText": {
"type": "string",
"value": "select `test_table`.`col1`, `test_table`.`col2`, `test_table`.`col4` from `stocks`.`test_table`"
},
"__createdBy": {
"type": "string",
"value": "anonymous"
},
"__typeName": {
"type": "string",
"value": "hive_table"
},
"__modificationTimestamp": {
"type": "long",
"value": 1522693838471
},
"Asset.owner": {
"type": "string",
"value": "anonymous"
},
"hive_table.lastAccessTime": {
"type": "long",
"value": 1522693834000
},
"hive_table.viewOriginalText": {
"type": "string",
"value": "select col1, col2, col4 from test_table"
},
"hive_table.columns": {
"type": "list",
"value": [{
"type": "string",
"value": "816u-35tc-ao0l-47so"
}, {
"type": "string",
"value": "82rq-35tc-ao0l-2glc"
}]
},
"__timestamp": {
"type": "long",
"value": 1522693835017
},
"_id": 147504,
"_type": "vertex"
}
......@@ -35,6 +35,7 @@ import com.tinkerpop.pipes.util.structures.Row;
import org.apache.atlas.AtlasErrorCode;
import org.apache.atlas.exception.AtlasBaseException;
import org.apache.atlas.groovy.GroovyExpression;
import org.apache.atlas.model.impexp.MigrationStatus;
import org.apache.atlas.repository.graphdb.AtlasEdge;
import org.apache.atlas.repository.graphdb.AtlasGraph;
import org.apache.atlas.repository.graphdb.AtlasGraphManagement;
......@@ -419,6 +420,11 @@ public class Titan0Graph implements AtlasGraph<Titan0Vertex, Titan0Edge> {
public void loadLegacyGraphSON(Map<String, String> relationshipCache, InputStream fs) throws AtlasBaseException {
}
@Override
public MigrationStatus getMigrationStatus() {
return new MigrationStatus();
}
public void addMultiProperties(Set<String> names) {
multiProperties.addAll(names);
}
......
......@@ -18,97 +18,58 @@
package org.apache.atlas.repository.impexp;
import com.google.common.cache.CacheBuilder;
import com.google.common.cache.CacheLoader;
import com.google.common.cache.LoadingCache;
import com.google.common.annotations.VisibleForTesting;
import org.apache.atlas.annotation.AtlasService;
import org.apache.atlas.model.impexp.MigrationStatus;
import org.apache.atlas.repository.Constants;
import org.apache.atlas.repository.graph.GraphHelper;
import org.apache.atlas.repository.graphdb.AtlasGraph;
import org.apache.atlas.repository.graphdb.AtlasVertex;
import org.apache.commons.configuration.Configuration;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import javax.inject.Inject;
import javax.inject.Singleton;
import java.util.Date;
import java.util.Iterator;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.TimeUnit;
@AtlasService
@Singleton
public class MigrationProgressService {
private static final Logger LOG = LoggerFactory.getLogger(MigrationProgressService.class);
public static final String MIGRATION_QUERY_CACHE_TTL = "atlas.migration.query.cache.ttlInSecs";
private static final String MIGRATION_STATUS_TYPE_NAME = "__MigrationStatus";
private static final String CURRENT_INDEX_PROPERTY = "currentIndex";
private static final String OPERATION_STATUS_PROPERTY = "operationStatus";
private static final String START_TIME_PROPERTY = "startTime";
private static final String END_TIME_PROPERTY = "endTime";
private static final String TOTAL_COUNT_PROPERTY = "totalCount";
private static final String MIGRATION_STATUS_KEY = "1";
@VisibleForTesting
static long DEFAULT_CACHE_TTL_IN_SECS = 30 * 1000; // 30 secs
private final long cacheValidity;
private final AtlasGraph graph;
private final MigrationStatus defaultStatus = new MigrationStatus();
private LoadingCache<String, MigrationStatus> cache;
private MigrationStatus cachedStatus;
private long cacheExpirationTime = 0;
@Inject
public MigrationProgressService(AtlasGraph graph) {
public MigrationProgressService(Configuration configuration, AtlasGraph graph) {
this.graph = graph;
this.cacheValidity = (configuration != null) ?
configuration.getLong(MIGRATION_QUERY_CACHE_TTL, DEFAULT_CACHE_TTL_IN_SECS) :
DEFAULT_CACHE_TTL_IN_SECS;
}
public MigrationStatus getStatus() {
try {
if (cache == null) {
initCache();
cache.get(MIGRATION_STATUS_KEY);
return fetchStatus();
}
if(cache.size() > 0) {
return cache.get(MIGRATION_STATUS_KEY);
private MigrationStatus fetchStatus() {
long currentTime = System.currentTimeMillis();
if(resetCache(currentTime)) {
cachedStatus = graph.getMigrationStatus();
}
return defaultStatus;
} catch (ExecutionException e) {
return defaultStatus;
}
return cachedStatus;
}
private void initCache() {
this.cache = CacheBuilder.newBuilder().refreshAfterWrite(30, TimeUnit.SECONDS).
build(new CacheLoader<String, MigrationStatus>() {
@Override
public MigrationStatus load(String key) {
try {
return from(fetchStatusVertex());
} catch (Exception e) {
LOG.error("Error retrieving status.", e);
return defaultStatus;
}
private boolean resetCache(long currentTime) {
boolean ret = cachedStatus == null || currentTime > cacheExpirationTime;
if(ret) {
cacheExpirationTime = currentTime + cacheValidity;
}
private MigrationStatus from(AtlasVertex vertex) {
if (vertex == null) {
return null;
}
MigrationStatus ms = new MigrationStatus();
ms.setStartTime(GraphHelper.getSingleValuedProperty(vertex, START_TIME_PROPERTY, Date.class));
ms.setEndTime(GraphHelper.getSingleValuedProperty(vertex, END_TIME_PROPERTY, Date.class));
ms.setCurrentIndex(GraphHelper.getSingleValuedProperty(vertex, CURRENT_INDEX_PROPERTY, Long.class));
ms.setOperationStatus(GraphHelper.getSingleValuedProperty(vertex, OPERATION_STATUS_PROPERTY, String.class));
ms.setTotalCount(GraphHelper.getSingleValuedProperty(vertex, TOTAL_COUNT_PROPERTY, Long.class));
return ms;
}
private AtlasVertex fetchStatusVertex() {
Iterator<AtlasVertex> itr = graph.query().has(Constants.ENTITY_TYPE_PROPERTY_KEY, MIGRATION_STATUS_TYPE_NAME).vertices().iterator();
return itr.hasNext() ? itr.next() : null;
}
});
return ret;
}
}
......@@ -120,10 +120,12 @@ public class DataMigrationService implements Service {
@VisibleForTesting
void processIncomingTypesDef(File typesDefFile) throws AtlasBaseException {
try {
AtlasImportResult result = new AtlasImportResult();
String jsonStr = FileUtils.readFileToString(typesDefFile);
AtlasTypesDef typesDef = AtlasType.fromJson(jsonStr, AtlasTypesDef.class);
ImportTypeDefProcessor processor = new ImportTypeDefProcessor(typeDefStore, typeRegistry);
processor.processTypes(typesDef, new AtlasImportResult());
processor.processTypes(typesDef, result);
LOG.info(" types migrated: {}", result.getMetrics());
} catch (IOException e) {
LOG.error("processIncomingTypesDef: Could not process file: {}! Imported data may not be usable.", typesDefFile.getName());
}
......
......@@ -43,11 +43,16 @@ public class HiveParititionTest extends MigrationBaseAsserts {
@Test
public void fileImporterTest() throws IOException, AtlasBaseException {
final int EXPECTED_TOTAL_COUNT = 140;
final int EXPECTED_DB_COUNT = 1;
final int EXPECTED_TABLE_COUNT = 2;
final int EXPECTED_COLUMN_COUNT = 7;
runFileImporter("parts_db");
assertPartitionKeyProperty(getVertex("hive_table", "t1"), 1);
assertPartitionKeyProperty(getVertex("hive_table", "tv1"), 1);
assertHiveVertices(1, 2, 7);
assertHiveVertices(EXPECTED_DB_COUNT, EXPECTED_TABLE_COUNT, EXPECTED_COLUMN_COUNT);
assertTypeCountNameGuid("hive_db", 1, "parts_db", "ae30d78b-51b4-42ab-9436-8d60c8f68b95");
assertTypeCountNameGuid("hive_process", 1, "", "");
......@@ -55,7 +60,7 @@ public class HiveParititionTest extends MigrationBaseAsserts {
assertEdges("hive_table", "t1", AtlasEdgeDirection.OUT, 1, 1, "hive_db_tables");
assertEdges("hive_table", "tv1", AtlasEdgeDirection.OUT, 1, 1, "hive_db_tables");
assertMigrationStatus(136);
assertMigrationStatus(EXPECTED_TOTAL_COUNT);
}
private void assertPartitionKeyProperty(AtlasVertex vertex, int expectedCount) {
......
......@@ -37,9 +37,14 @@ public class HiveStocksTest extends MigrationBaseAsserts {
@Test
public void migrateStocks() throws AtlasBaseException, IOException {
final int EXPECTED_TOTAL_COUNT = 187;
final int EXPECTED_DB_COUNT = 1;
final int EXPECTED_TABLE_COUNT = 1;
final int EXPECTED_COLUMN_COUNT = 7;
runFileImporter("stocks_db");
assertHiveVertices(1, 1, 7);
assertHiveVertices(EXPECTED_DB_COUNT, EXPECTED_TABLE_COUNT, EXPECTED_COLUMN_COUNT);
assertTypeCountNameGuid("hive_db", 1, "stocks", "4e13b36b-9c54-4616-9001-1058221165d0");
assertTypeCountNameGuid("hive_table", 1, "stocks_daily", "5cfc2540-9947-40e0-8905-367e07481774");
assertTypeAttribute("hive_table", 7, "stocks_daily", "5cfc2540-9947-40e0-8905-367e07481774", "hive_table.columns");
......@@ -58,6 +63,6 @@ public class HiveStocksTest extends MigrationBaseAsserts {
assertEdges(getVertex("hive_table", "stocks_daily").getEdges(AtlasEdgeDirection.OUT).iterator(), 1, 1, "hive_db_tables");
assertEdges(getVertex("hive_column", "high").getEdges(AtlasEdgeDirection.OUT).iterator(), 1,1, "hive_table_columns");
assertMigrationStatus(187);
assertMigrationStatus(EXPECTED_TOTAL_COUNT);
}
}
......@@ -40,7 +40,7 @@ import static org.apache.atlas.graph.GraphSandboxUtil.useLocalSolr;
import static org.apache.atlas.repository.impexp.ZipFileResourceTestUtils.loadModelFromJson;
import static org.testng.Assert.assertEquals;
import static org.testng.Assert.assertNotNull;
import static org.testng.AssertJUnit.assertTrue;
import static org.testng.Assert.assertTrue;
public class MigrationBaseAsserts {
protected static final String ASSERT_NAME_PROPERTY = "Asset.name";
......@@ -76,6 +76,7 @@ public class MigrationBaseAsserts {
private void loadTypesFromJson() throws IOException, AtlasBaseException {
loadModelFromJson("0000-Area0/0010-base_model.json", typeDefStore, typeRegistry);
loadModelFromJson("1000-Hadoop/1020-fs_model.json", typeDefStore, typeRegistry);
loadModelFromJson("1000-Hadoop/1030-hive_model.json", typeDefStore, typeRegistry);
}
......
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS 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.repository.migration;
import org.apache.atlas.model.impexp.MigrationStatus;
import org.apache.atlas.repository.graphdb.*;
import org.apache.atlas.repository.graphdb.janus.migration.ReaderStatusManager;
import org.apache.atlas.repository.impexp.MigrationProgressService;
import org.apache.commons.configuration.Configuration;
import org.apache.commons.lang.StringUtils;
import org.apache.tinkerpop.gremlin.tinkergraph.structure.TinkerGraph;
import org.testng.annotations.Test;
import static org.mockito.Matchers.anyLong;
import static org.mockito.Matchers.anyString;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;
import static org.testng.Assert.*;
public class MigrationProgressServiceTest {
private final long currentIndex = 100l;
private final long totalIndex = 1000l;
private final long increment = 1001l;
private final String statusSuccess = ReaderStatusManager.STATUS_SUCCESS;
private static class AtlasTinkerGraph {
public static AtlasGraph create(TinkerGraph tg) {
AtlasGraph g = mock(AtlasGraph.class);
when(g.getMigrationStatus()).thenAnswer(invocation -> ReaderStatusManager.get(tg));
return g;
}
public static AtlasGraph create() {
return create(TinkerGraph.open());
}
}
@Test
public void absentStatusNodeReturnsDefaultStatus() {
MigrationProgressService mps = getMigrationStatusForTest(null, null);
MigrationStatus ms = mps.getStatus();
assertNotNull(ms);
assertTrue(StringUtils.isEmpty(ms.getOperationStatus()));
assertEquals(ms.getCurrentIndex(), 0);
assertEquals(ms.getTotalCount(), 0);
}
@Test
public void existingStatusNodeRetrurnStatus() {
final long currentIndex = 100l;
final long totalIndex = 1000l;
final String status = ReaderStatusManager.STATUS_SUCCESS;
TinkerGraph tg = createUpdateStatusNode(null, currentIndex, totalIndex, status);
MigrationProgressService mps = getMigrationStatusForTest(null, tg);
MigrationStatus ms = mps.getStatus();
assertMigrationStatus(totalIndex, status, ms);
}
@Test
public void cachedStatusReturnedIfQueriedBeforeCacheExpiration() {
TinkerGraph tg = createUpdateStatusNode(null, currentIndex, totalIndex, statusSuccess);
MigrationProgressService mps = getMigrationStatusForTest(null, tg);
MigrationStatus ms = mps.getStatus();
createUpdateStatusNode(tg, currentIndex + increment, totalIndex + increment, ReaderStatusManager.STATUS_FAILED);
MigrationStatus ms2 = mps.getStatus();
assertEquals(ms.hashCode(), ms2.hashCode());
assertMigrationStatus(totalIndex, statusSuccess, ms);
}
private MigrationProgressService getMigrationStatusForTest(Configuration cfg, TinkerGraph tg) {
return new MigrationProgressService(cfg, AtlasTinkerGraph.create(tg));
}
@Test
public void cachedUpdatedIfQueriedAfterCacheExpiration() throws InterruptedException {
final String statusFailed = ReaderStatusManager.STATUS_FAILED;
TinkerGraph tg = createUpdateStatusNode(null, currentIndex, totalIndex, statusSuccess);
long cacheTTl = 100l;
MigrationProgressService mps = getMigrationStatusForTest(getStubConfiguration(cacheTTl), tg);
MigrationStatus ms = mps.getStatus();
assertMigrationStatus(totalIndex, statusSuccess, ms);
createUpdateStatusNode(tg, currentIndex + increment, totalIndex + increment, ReaderStatusManager.STATUS_FAILED);
Thread.sleep(2 * cacheTTl);
MigrationStatus ms2 = mps.getStatus();
assertNotEquals(ms.hashCode(), ms2.hashCode());
assertMigrationStatus(totalIndex + increment, statusFailed, ms2);
}
private Configuration getStubConfiguration(long ttl) {
Configuration cfg = mock(Configuration.class);
when(cfg.getLong(anyString(), anyLong())).thenReturn(ttl);
return cfg;
}
private TinkerGraph createUpdateStatusNode(TinkerGraph tg, long currentIndex, long totalIndex, String status) {
if(tg == null) {
tg = TinkerGraph.open();
}
ReaderStatusManager rsm = new ReaderStatusManager(tg, tg);
rsm.update(tg, currentIndex);
rsm.end(tg, totalIndex, status);
return tg;
}
private void assertMigrationStatus(long totalIndex, String status, MigrationStatus ms) {
assertNotNull(ms);
assertEquals(ms.getOperationStatus(), status);
assertEquals(ms.getCurrentIndex(), totalIndex);
assertEquals(ms.getTotalCount(), totalIndex);
}
}
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.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.repository.migration;
import com.google.inject.Inject;
import org.apache.atlas.TestModules;
import org.apache.atlas.exception.AtlasBaseException;
import org.apache.atlas.repository.graph.GraphHelper;
import org.apache.atlas.repository.graphdb.AtlasGraph;
import org.apache.atlas.repository.graphdb.AtlasVertex;
import org.apache.atlas.type.AtlasBuiltInTypes;
import org.testng.annotations.Guice;
import org.testng.annotations.Test;
import java.io.IOException;
import java.math.BigDecimal;
import java.math.BigInteger;
import java.util.Iterator;
import static org.testng.Assert.assertEquals;
import static org.testng.Assert.assertNotNull;
@Guice(modules = TestModules.TestOnlyModule.class)
public class PathTest extends MigrationBaseAsserts {
@Inject
public PathTest(AtlasGraph graph) {
super(graph);
}
@Test
public void migrationImport() throws IOException, AtlasBaseException {
runFileImporter("path_db");
AtlasVertex v = assertHdfsPathVertices(1);
assertVertexProperties(v);
assertMigrationStatus(88);
}
private void assertVertexProperties(AtlasVertex v) {
final String HASH_CODE_PROPERTY = "hdfs_path.hashCode";
final String RETENTION_PROPERTY = "hdfs_path.retention";
AtlasBuiltInTypes.AtlasBigIntegerType bitRef = new AtlasBuiltInTypes.AtlasBigIntegerType();
AtlasBuiltInTypes.AtlasBigDecimalType bdtRef = new AtlasBuiltInTypes.AtlasBigDecimalType();
BigInteger bitExpected = bitRef.getNormalizedValue(612361213421234L);
BigDecimal bdtExpected = bdtRef.getNormalizedValue(125353);
BigInteger bit = GraphHelper.getSingleValuedProperty(v, HASH_CODE_PROPERTY, BigInteger.class);
BigDecimal bdt = GraphHelper.getSingleValuedProperty(v, RETENTION_PROPERTY, BigDecimal.class);
assertEquals(bit, bitExpected);
assertEquals(bdt.compareTo(bdtExpected), 0);
}
protected AtlasVertex assertHdfsPathVertices(int expectedCount) {
int i = 0;
AtlasVertex vertex = null;
Iterator<AtlasVertex> results = getVertices("hdfs_path", null);
for (Iterator<AtlasVertex> it = results; it.hasNext(); i++) {
vertex = it.next();
assertNotNull(vertex);
}
assertEquals(i, expectedCount);
return vertex;
}
}
......@@ -29,6 +29,7 @@ import org.apache.atlas.store.AtlasTypeDefStore;
import org.apache.atlas.type.AtlasRelationshipType;
import org.apache.atlas.type.AtlasTypeRegistry;
import org.apache.commons.lang.StringUtils;
import org.apache.tinkerpop.gremlin.tinkergraph.structure.TinkerGraph;
import org.jcodings.util.Hash;
import org.testng.annotations.BeforeClass;
import org.testng.annotations.Guice;
......
{
"mode": "EXTENDED",
"vertices": [{
"__type.file_action.READ_EXECUTE": {
"type": "integer",
"value": 5
},
"__type.name": {
"type": "string",
"value": "file_action"
},
"__type.file_action.NONE": {
"type": "integer",
"value": 0
},
"__type.file_action.ALL": {
"type": "integer",
"value": 7
},
"__type.file_action.READ_WRITE": {
"type": "integer",
"value": 6
},
"__modifiedBy": {
"type": "string",
"value": "root"
},
"__type.options": {
"type": "string",
"value": "null"
},
"__guid": {
"type": "string",
"value": "ee607a83-e4ac-4a93-8bdb-cbae7b89c056"
},
"__type.file_action.WRITE_EXECUTE": {
"type": "integer",
"value": 3
},
"__version": {
"type": "long",
"value": 1
},
"__type.file_action.EXECUTE": {
"type": "integer",
"value": 1
},
"__type.category": {
"type": "string",
"value": "ENUM"
},
"__type.version": {
"type": "string",
"value": "1.0"
},
"__type.file_action": {
"type": "list",
"value": [{
"type": "string",
"value": "NONE"
}, {
"type": "string",
"value": "EXECUTE"
}, {
"type": "string",
"value": "WRITE"
}, {
"type": "string",
"value": "WRITE_EXECUTE"
}, {
"type": "string",
"value": "READ"
}, {
"type": "string",
"value": "READ_EXECUTE"
}, {
"type": "string",
"value": "READ_WRITE"
}, {
"type": "string",
"value": "ALL"
}]
},
"__createdBy": {
"type": "string",
"value": "root"
},
"__modificationTimestamp": {
"type": "long",
"value": 1522707673425
},
"__type": {
"type": "string",
"value": "typeSystem"
},
"__type.file_action.READ": {
"type": "integer",
"value": 4
},
"__type.description": {
"type": "string",
"value": "file_action"
},
"__timestamp": {
"type": "long",
"value": 1522707673425
},
"__type.file_action.WRITE": {
"type": "integer",
"value": 2
},
"_id": 16488,
"_type": "vertex"
}, {
"__type.hive_process.operationType": {
"type": "string",
"value": "{\"multiplicity\":\"{\\\"lower\\\":1,\\\"upper\\\":1,\\\"isUnique\\\":false}\",\"isIndexable\":true,\"isComposite\":false,\"reverseAttributeName\":null,\"dataType\":\"string\",\"name\":\"operationType\",\"isUnique\":false}"
},
"__type.name": {
"type": "string",
"value": "hive_process"
},
"__type.hive_process.userName": {
"type": "string",
"value": "{\"multiplicity\":\"{\\\"lower\\\":1,\\\"upper\\\":1,\\\"isUnique\\\":false}\",\"isIndexable\":true,\"isComposite\":false,\"reverseAttributeName\":null,\"dataType\":\"string\",\"name\":\"userName\",\"isUnique\":false}"
},
"__modifiedBy": {
"type": "string",
"value": "root"
},
"__type.hive_process.queryId": {
"type": "string",
"value": "{\"multiplicity\":\"{\\\"lower\\\":1,\\\"upper\\\":1,\\\"isUnique\\\":false}\",\"isIndexable\":false,\"isComposite\":false,\"reverseAttributeName\":null,\"dataType\":\"string\",\"name\":\"queryId\",\"isUnique\":false}"
},
"__type.hive_process.queryPlan": {
"type": "string",
"value": "{\"multiplicity\":\"{\\\"lower\\\":1,\\\"upper\\\":1,\\\"isUnique\\\":false}\",\"isIndexable\":false,\"isComposite\":false,\"reverseAttributeName\":null,\"dataType\":\"string\",\"name\":\"queryPlan\",\"isUnique\":false}"
},
"__type.options": {
"type": "string",
"value": "null"
},
"__type.hive_process.recentQueries": {
"type": "string",
"value": "{\"multiplicity\":\"{\\\"lower\\\":0,\\\"upper\\\":1,\\\"isUnique\\\":false}\",\"isIndexable\":false,\"isComposite\":false,\"reverseAttributeName\":null,\"dataType\":\"array<string>\",\"name\":\"recentQueries\",\"isUnique\":false}"
},
"__guid": {
"type": "string",
"value": "51d50b18-31fe-49b3-ba53-cfbcef964241"
},
"__version": {
"type": "long",
"value": 1
},
"__type.category": {
"type": "string",
"value": "CLASS"
},
"__type.hive_process": {
"type": "list",
"value": [{
"type": "string",
"value": "startTime"
}, {
"type": "string",
"value": "endTime"
}, {
"type": "string",
"value": "userName"
}, {
"type": "string",
"value": "operationType"
}, {
"type": "string",
"value": "queryText"
}, {
"type": "string",
"value": "queryPlan"
}, {
"type": "string",
"value": "queryId"
}, {
"type": "string",
"value": "recentQueries"
}, {
"type": "string",
"value": "clusterName"
}, {
"type": "string",
"value": "queryGraph"
}]
},
"__type.version": {
"type": "string",
"value": "1.0"
},
"__createdBy": {
"type": "string",
"value": "root"
},
"__type.hive_process.clusterName": {
"type": "string",
"value": "{\"multiplicity\":\"{\\\"lower\\\":0,\\\"upper\\\":1,\\\"isUnique\\\":false}\",\"isIndexable\":false,\"isComposite\":false,\"reverseAttributeName\":null,\"dataType\":\"string\",\"name\":\"clusterName\",\"isUnique\":false}"
},
"__modificationTimestamp": {
"type": "long",
"value": 1522707675692
},
"__type": {
"type": "string",
"value": "typeSystem"
},
"__type.hive_process.endTime": {
"type": "string",
"value": "{\"multiplicity\":\"{\\\"lower\\\":1,\\\"upper\\\":1,\\\"isUnique\\\":false}\",\"isIndexable\":false,\"isComposite\":false,\"reverseAttributeName\":null,\"dataType\":\"date\",\"name\":\"endTime\",\"isUnique\":false}"
},
"__type.hive_process.queryText": {
"type": "string",
"value": "{\"multiplicity\":\"{\\\"lower\\\":1,\\\"upper\\\":1,\\\"isUnique\\\":false}\",\"isIndexable\":false,\"isComposite\":false,\"reverseAttributeName\":null,\"dataType\":\"string\",\"name\":\"queryText\",\"isUnique\":false}"
},
"__type.hive_process.queryGraph": {
"type": "string",
"value": "{\"multiplicity\":\"{\\\"lower\\\":0,\\\"upper\\\":1,\\\"isUnique\\\":false}\",\"isIndexable\":false,\"isComposite\":false,\"reverseAttributeName\":null,\"dataType\":\"string\",\"name\":\"queryGraph\",\"isUnique\":false}"
},
"__type.hive_process.startTime": {
"type": "string",
"value": "{\"multiplicity\":\"{\\\"lower\\\":1,\\\"upper\\\":1,\\\"isUnique\\\":false}\",\"isIndexable\":false,\"isComposite\":false,\"reverseAttributeName\":null,\"dataType\":\"date\",\"name\":\"startTime\",\"isUnique\":false}"
},
"__type.description": {
"type": "string",
"value": "hive_process"
},
"__timestamp": {
"type": "long",
"value": 1522707675692
},
"_id": 32872,
"_type": "vertex"
}, {
"__type.name": {
"type": "string",
"value": "TaxonomyTerm"
},
"__type.TaxonomyTerm.atlas.taxonomy": {
"type": "string",
"value": "{\"multiplicity\":\"{\\\"lower\\\":0,\\\"upper\\\":1,\\\"isUnique\\\":false}\",\"isIndexable\":false,\"isComposite\":false,\"reverseAttributeName\":null,\"dataType\":\"string\",\"name\":\"atlas.taxonomy\",\"isUnique\":false}"
},
"__modifiedBy": {
"type": "string",
"value": "root"
},
"__type.options": {
"type": "string",
"value": "null"
},
"__guid": {
"type": "string",
"value": "1732545c-9e0e-45ec-95aa-09c04f0b2a63"
},
"__version": {
"type": "long",
"value": 1
},
"__type.TaxonomyTerm": {
"type": "list",
"value": [{
"type": "string",
"value": "atlas.taxonomy"
}]
},
"__type.category": {
"type": "string",
"value": "TRAIT"
},
"__type.version": {
"type": "string",
"value": "1.0"
},
"__createdBy": {
"type": "string",
"value": "root"
},
"__modificationTimestamp": {
"type": "long",
"value": 1522707667929
},
"__type": {
"type": "string",
"value": "typeSystem"
},
"__type.description": {
"type": "string",
"value": "TaxonomyTerm"
},
"__timestamp": {
"type": "long",
"value": 1522707667929
},
"_id": 16512,
"_type": "vertex"
}, {
"__type.name": {
"type": "string",
"value": "hive_serde"
},
"__type.hive_serde.parameters": {
"type": "string",
"value": "{\"multiplicity\":\"{\\\"lower\\\":0,\\\"upper\\\":1,\\\"isUnique\\\":false}\",\"isIndexable\":false,\"isComposite\":false,\"reverseAttributeName\":null,\"dataType\":\"map<string,string>\",\"name\":\"parameters\",\"isUnique\":false}"
},
"__modifiedBy": {
"type": "string",
"value": "root"
},
"__type.options": {
"type": "string",
"value": "null"
},
"__type.hive_serde.serializationLib": {
"type": "string",
"value": "{\"multiplicity\":\"{\\\"lower\\\":0,\\\"upper\\\":1,\\\"isUnique\\\":false}\",\"isIndexable\":false,\"isComposite\":false,\"reverseAttributeName\":null,\"dataType\":\"string\",\"name\":\"serializationLib\",\"isUnique\":false}"
},
"__guid": {
"type": "string",
"value": "29e2774b-fc9f-44f1-adf9-f7bdc7db1a70"
},
"__version": {
"type": "long",
"value": 1
},
"__type.hive_serde": {
"type": "list",
"value": [{
"type": "string",
"value": "name"
}, {
"type": "string",
"value": "serializationLib"
}, {
"type": "string",
"value": "parameters"
}]
},
"__type.category": {
"type": "string",
"value": "STRUCT"
},
"__type.version": {
"type": "string",
"value": "1.0"
},
"__createdBy": {
"type": "string",
"value": "root"
},
"__modificationTimestamp": {
"type": "long",
"value": 1522707675680
},
"__type": {
"type": "string",
"value": "typeSystem"
},
"__type.hive_serde.name": {
"type": "string",
"value": "{\"multiplicity\":\"{\\\"lower\\\":0,\\\"upper\\\":1,\\\"isUnique\\\":false}\",\"isIndexable\":false,\"isComposite\":false,\"reverseAttributeName\":null,\"dataType\":\"string\",\"name\":\"name\",\"isUnique\":false}"
},
"__type.description": {
"type": "string",
"value": "hive_serde"
},
"__timestamp": {
"type": "long",
"value": 1522707675680
},
"_id": 16520,
"_type": "vertex"
}, {
"__type.name": {
"type": "string",
"value": "__internal"
},
"__version": {
"type": "long",
"value": 1
},
"__type.category": {
"type": "string",
"value": "CLASS"
},
"__modifiedBy": {
"type": "string",
"value": "root"
},
"__type.version": {
"type": "string",
"value": "1.0"
},
"__createdBy": {
"type": "string",
"value": "root"
},
"__modificationTimestamp": {
"type": "long",
"value": 1522707669396
},
"__type": {
"type": "string",
"value": "typeSystem"
},
"__type.options": {
"type": "string",
"value": "null"
},
"__type.description": {
"type": "string",
"value": "__internal"
},
"__guid": {
"type": "string",
"value": "358d2e1d-7a63-496f-82fb-8327376ba2de"
},
"__timestamp": {
"type": "long",
"value": 1522707669396
},
"_id": 16544,
"_type": "vertex"
}, {
"__type.fs_permissions.group": {
"type": "string",
"value": "{\"multiplicity\":\"{\\\"lower\\\":1,\\\"upper\\\":1,\\\"isUnique\\\":false}\",\"isIndexable\":true,\"isComposite\":false,\"reverseAttributeName\":null,\"dataType\":\"file_action\",\"name\":\"group\",\"isUnique\":false}"
},
"__type.name": {
"type": "string",
"value": "fs_permissions"
},
"__modifiedBy": {
"type": "string",
"value": "root"
},
"__type.fs_permissions.sticky": {
"type": "string",
"value": "{\"multiplicity\":\"{\\\"lower\\\":1,\\\"upper\\\":1,\\\"isUnique\\\":false}\",\"isIndexable\":true,\"isComposite\":false,\"reverseAttributeName\":null,\"dataType\":\"boolean\",\"name\":\"sticky\",\"isUnique\":false}"
},
"__type.options": {
"type": "string",
"value": "null"
},
"__type.fs_permissions.others": {
"type": "string",
"value": "{\"multiplicity\":\"{\\\"lower\\\":1,\\\"upper\\\":1,\\\"isUnique\\\":false}\",\"isIndexable\":true,\"isComposite\":false,\"reverseAttributeName\":null,\"dataType\":\"file_action\",\"name\":\"others\",\"isUnique\":false}"
},
"__guid": {
"type": "string",
"value": "24f03770-fb23-4efe-98c0-65ed26816237"
},
"__version": {
"type": "long",
"value": 1
},
"__type.fs_permissions.user": {
"type": "string",
"value": "{\"multiplicity\":\"{\\\"lower\\\":1,\\\"upper\\\":1,\\\"isUnique\\\":false}\",\"isIndexable\":true,\"isComposite\":false,\"reverseAttributeName\":null,\"dataType\":\"file_action\",\"name\":\"user\",\"isUnique\":false}"
},
"__type.category": {
"type": "string",
"value": "STRUCT"
},
"__type.version": {
"type": "string",
"value": "1.0"
},
"__createdBy": {
"type": "string",
"value": "root"
},
"__modificationTimestamp": {
"type": "long",
"value": 1522707673475
},
"__type": {
"type": "string",
"value": "typeSystem"
},
"__type.fs_permissions": {
"type": "list",
"value": [{
"type": "string",
"value": "group"
}, {
"type": "string",
"value": "user"
}, {
"type": "string",
"value": "sticky"
}, {
"type": "string",
"value": "others"
}]
},
"__type.description": {
"type": "string",
"value": "fs_permissions"
},
"__timestamp": {
"type": "long",
"value": 1522707673475
},
"_id": 32928,
"_type": "vertex"
}, {
"__type.name": {
"type": "string",
"value": "hive_principal_type"
},
"__type.hive_principal_type.ROLE": {
"type": "integer",
"value": 2
},
"__modifiedBy": {
"type": "string",
"value": "root"
},
"__type.options": {
"type": "string",
"value": "null"
},
"__type.hive_principal_type.USER": {
"type": "integer",
"value": 1
},
"__guid": {
"type": "string",
"value": "560cdc45-5883-448f-9217-eafdaf577bf5"
},
"__type.hive_principal_type.GROUP": {
"type": "integer",
"value": 3
},
"__version": {
"type": "long",
"value": 1
},
"__type.hive_principal_type": {
"type": "list",
"value": [{
"type": "string",
"value": "USER"
}, {
"type": "string",
"value": "ROLE"
}, {
"type": "string",
"value": "GROUP"
}]
},
"__type.category": {
"type": "string",
"value": "ENUM"
},
"__type.version": {
"type": "string",
"value": "1.0"
},
"__createdBy": {
"type": "string",
"value": "root"
},
"__modificationTimestamp": {
"type": "long",
"value": 1522707674715
},
"__type": {
"type": "string",
"value": "typeSystem"
},
"__type.description": {
"type": "string",
"value": "hive_principal_type"
},
"__timestamp": {
"type": "long",
"value": 1522707674715
},
"_id": 49312,
"_type": "vertex"
}, {
"__type.name": {
"type": "string",
"value": "Process"
},
"__modifiedBy": {
"type": "string",
"value": "root"
},
"__type.options": {
"type": "string",
"value": "null"
},
"__type.Process.inputs": {
"type": "string",
"value": "{\"multiplicity\":\"{\\\"lower\\\":0,\\\"upper\\\":1,\\\"isUnique\\\":false}\",\"isIndexable\":false,\"isComposite\":false,\"reverseAttributeName\":null,\"dataType\":\"array<DataSet>\",\"name\":\"inputs\",\"isUnique\":false}"
},
"__guid": {
"type": "string",
"value": "89d95a88-a996-45cc-a1c0-2536bd99a68c"
},
"__version": {
"type": "long",
"value": 1
},
"__type.category": {
"type": "string",
"value": "CLASS"
},
"__type.version": {
"type": "string",
"value": "1.0"
},
"__createdBy": {
"type": "string",
"value": "root"
},
"__modificationTimestamp": {
"type": "long",
"value": 1522707671495
},
"__type": {
"type": "string",
"value": "typeSystem"
},
"__type.Process": {
"type": "list",
"value": [{
"type": "string",
"value": "inputs"
}, {
"type": "string",
"value": "outputs"
}]
},
"__type.Process.outputs": {
"type": "string",
"value": "{\"multiplicity\":\"{\\\"lower\\\":0,\\\"upper\\\":1,\\\"isUnique\\\":false}\",\"isIndexable\":false,\"isComposite\":false,\"reverseAttributeName\":null,\"dataType\":\"array<DataSet>\",\"name\":\"outputs\",\"isUnique\":false}"
},
"__type.description": {
"type": "string",
"value": "Process"
},
"__timestamp": {
"type": "long",
"value": 1522707671495
},
"_id": 16560,
"_type": "vertex"
}, {
"__type.name": {
"type": "string",
"value": "hive_column_lineage"
},
"__modifiedBy": {
"type": "string",
"value": "root"
},
"__type.options": {
"type": "string",
"value": "null"
},
"__type.hive_column_lineage.expression": {
"type": "string",
"value": "{\"multiplicity\":\"{\\\"lower\\\":0,\\\"upper\\\":1,\\\"isUnique\\\":false}\",\"isIndexable\":false,\"isComposite\":false,\"reverseAttributeName\":null,\"dataType\":\"string\",\"name\":\"expression\",\"isUnique\":false}"
},
"__guid": {
"type": "string",
"value": "688cdf4c-c187-4ed7-8236-e9067005ab5d"
},
"__version": {
"type": "long",
"value": 1
},
"__type.hive_column_lineage": {
"type": "list",
"value": [{
"type": "string",
"value": "query"
}, {
"type": "string",
"value": "depenendencyType"
}, {
"type": "string",
"value": "expression"
}]
},
"__type.category": {
"type": "string",
"value": "CLASS"
},
"__type.version": {
"type": "string",
"value": "1.0"
},
"__createdBy": {
"type": "string",
"value": "root"
},
"__modificationTimestamp": {
"type": "long",
"value": 1522707676486
},
"__type": {
"type": "string",
"value": "typeSystem"
},
"__type.hive_column_lineage.depenendencyType": {
"type": "string",
"value": "{\"multiplicity\":\"{\\\"lower\\\":1,\\\"upper\\\":1,\\\"isUnique\\\":false}\",\"isIndexable\":false,\"isComposite\":false,\"reverseAttributeName\":null,\"dataType\":\"string\",\"name\":\"depenendencyType\",\"isUnique\":false}"
},
"__type.hive_column_lineage.query": {
"type": "string",
"value": "{\"multiplicity\":\"{\\\"lower\\\":1,\\\"upper\\\":1,\\\"isUnique\\\":false}\",\"isIndexable\":false,\"isComposite\":false,\"reverseAttributeName\":null,\"dataType\":\"hive_process\",\"name\":\"query\",\"isUnique\":false}"
},
"__type.description": {
"type": "string",
"value": "hive_column_lineage"
},
"__timestamp": {
"type": "long",
"value": 1522707676486
},
"_id": 32944,
"_type": "vertex"
}, {
"__type.name": {
"type": "string",
"value": "hive_table"
},
"__modifiedBy": {
"type": "string",
"value": "root"
},
"__type.hive_table.tableType": {
"type": "string",
"value": "{\"multiplicity\":\"{\\\"lower\\\":0,\\\"upper\\\":1,\\\"isUnique\\\":false}\",\"isIndexable\":false,\"isComposite\":false,\"reverseAttributeName\":null,\"dataType\":\"string\",\"name\":\"tableType\",\"isUnique\":false}"
},
"__type.hive_table.retention": {
"type": "string",
"value": "{\"multiplicity\":\"{\\\"lower\\\":0,\\\"upper\\\":1,\\\"isUnique\\\":false}\",\"isIndexable\":false,\"isComposite\":false,\"reverseAttributeName\":null,\"dataType\":\"int\",\"name\":\"retention\",\"isUnique\":false}"
},
"__version": {
"type": "long",
"value": 2
},
"__type.hive_table.db": {
"type": "string",
"value": "{\"multiplicity\":\"{\\\"lower\\\":1,\\\"upper\\\":1,\\\"isUnique\\\":false}\",\"isIndexable\":false,\"isComposite\":false,\"reverseAttributeName\":null,\"dataType\":\"hive_db\",\"name\":\"db\",\"isUnique\":false}"
},
"__type.hive_table.viewOriginalText": {
"type": "string",
"value": "{\"multiplicity\":\"{\\\"lower\\\":0,\\\"upper\\\":1,\\\"isUnique\\\":false}\",\"isIndexable\":false,\"isComposite\":false,\"reverseAttributeName\":null,\"dataType\":\"string\",\"name\":\"viewOriginalText\",\"isUnique\":false}"
},
"__type.version": {
"type": "string",
"value": "1.1"
},
"__type": {
"type": "string",
"value": "typeSystem"
},
"__type.hive_table.partitionKeys": {
"type": "string",
"value": "{\"multiplicity\":\"{\\\"lower\\\":0,\\\"upper\\\":1,\\\"isUnique\\\":false}\",\"isIndexable\":false,\"isComposite\":true,\"reverseAttributeName\":null,\"dataType\":\"array<hive_column>\",\"name\":\"partitionKeys\",\"isUnique\":false}"
},
"__timestamp": {
"type": "long",
"value": 1522707676395
},
"__type.hive_table.aliases": {
"type": "string",
"value": "{\"multiplicity\":\"{\\\"lower\\\":0,\\\"upper\\\":1,\\\"isUnique\\\":false}\",\"isIndexable\":false,\"isComposite\":false,\"reverseAttributeName\":null,\"dataType\":\"array<string>\",\"name\":\"aliases\",\"isUnique\":false}"
},
"__type.hive_table.sd": {
"type": "string",
"value": "{\"multiplicity\":\"{\\\"lower\\\":0,\\\"upper\\\":1,\\\"isUnique\\\":false}\",\"isIndexable\":false,\"isComposite\":true,\"reverseAttributeName\":null,\"dataType\":\"hive_storagedesc\",\"name\":\"sd\",\"isUnique\":false}"
},
"__type.hive_table.columns": {
"type": "string",
"value": "{\"multiplicity\":\"{\\\"lower\\\":0,\\\"upper\\\":1,\\\"isUnique\\\":false}\",\"isIndexable\":false,\"isComposite\":true,\"reverseAttributeName\":null,\"dataType\":\"array<hive_column>\",\"name\":\"columns\",\"isUnique\":false}"
},
"__type.hive_table.temporary": {
"type": "string",
"value": "{\"multiplicity\":\"{\\\"lower\\\":0,\\\"upper\\\":1,\\\"isUnique\\\":false}\",\"isIndexable\":true,\"isComposite\":false,\"reverseAttributeName\":null,\"dataType\":\"boolean\",\"name\":\"temporary\",\"isUnique\":false}"
},
"__type.options": {
"type": "string",
"value": "{\"schemaElementsAttribute\":\"columns\"}"
},
"__type.hive_table.comment": {
"type": "string",
"value": "{\"multiplicity\":\"{\\\"lower\\\":0,\\\"upper\\\":1,\\\"isUnique\\\":false}\",\"isIndexable\":false,\"isComposite\":false,\"reverseAttributeName\":null,\"dataType\":\"string\",\"name\":\"comment\",\"isUnique\":false}"
},
"__type.hive_table.viewExpandedText": {
"type": "string",
"value": "{\"multiplicity\":\"{\\\"lower\\\":0,\\\"upper\\\":1,\\\"isUnique\\\":false}\",\"isIndexable\":false,\"isComposite\":false,\"reverseAttributeName\":null,\"dataType\":\"string\",\"name\":\"viewExpandedText\",\"isUnique\":false}"
},
"__guid": {
"type": "string",
"value": "2233745d-f770-4e13-8fd7-75ea5254f221"
},
"__type.hive_table.createTime": {
"type": "string",
"value": "{\"multiplicity\":\"{\\\"lower\\\":0,\\\"upper\\\":1,\\\"isUnique\\\":false}\",\"isIndexable\":false,\"isComposite\":false,\"reverseAttributeName\":null,\"dataType\":\"date\",\"name\":\"createTime\",\"isUnique\":false}"
},
"__type.hive_table.lastAccessTime": {
"type": "string",
"value": "{\"multiplicity\":\"{\\\"lower\\\":0,\\\"upper\\\":1,\\\"isUnique\\\":false}\",\"isIndexable\":false,\"isComposite\":false,\"reverseAttributeName\":null,\"dataType\":\"date\",\"name\":\"lastAccessTime\",\"isUnique\":false}"
},
"__type.hive_table": {
"type": "list",
"value": [{
"type": "string",
"value": "db"
}, {
"type": "string",
"value": "createTime"
}, {
"type": "string",
"value": "lastAccessTime"
}, {
"type": "string",
"value": "comment"
}, {
"type": "string",
"value": "retention"
}, {
"type": "string",
"value": "sd"
}, {
"type": "string",
"value": "partitionKeys"
}, {
"type": "string",
"value": "aliases"
}, {
"type": "string",
"value": "columns"
}, {
"type": "string",
"value": "parameters"
}, {
"type": "string",
"value": "viewOriginalText"
}, {
"type": "string",
"value": "viewExpandedText"
}, {
"type": "string",
"value": "tableType"
}, {
"type": "string",
"value": "temporary"
}]
},
"__type.hive_table.parameters": {
"type": "string",
"value": "{\"multiplicity\":\"{\\\"lower\\\":0,\\\"upper\\\":1,\\\"isUnique\\\":false}\",\"isIndexable\":false,\"isComposite\":false,\"reverseAttributeName\":null,\"dataType\":\"map<string,string>\",\"name\":\"parameters\",\"isUnique\":false}"
},
"__type.category": {
"type": "string",
"value": "CLASS"
},
"__createdBy": {
"type": "string",
"value": "root"
},
"__modificationTimestamp": {
"type": "long",
"value": 1522707677388
},
"__type.description": {
"type": "string",
"value": "hive_table"
},
"_id": 16696,
"_type": "vertex"
}, {
"__type.name": {
"type": "string",
"value": "hive_db"
},
"__type.hive_db.parameters": {
"type": "string",
"value": "{\"multiplicity\":\"{\\\"lower\\\":0,\\\"upper\\\":1,\\\"isUnique\\\":false}\",\"isIndexable\":false,\"isComposite\":false,\"reverseAttributeName\":null,\"dataType\":\"map<string,string>\",\"name\":\"parameters\",\"isUnique\":false}"
},
"__modifiedBy": {
"type": "string",
"value": "root"
},
"__type.hive_db": {
"type": "list",
"value": [{
"type": "string",
"value": "clusterName"
}, {
"type": "string",
"value": "location"
}, {
"type": "string",
"value": "parameters"
}, {
"type": "string",
"value": "ownerType"
}]
},
"__type.options": {
"type": "string",
"value": "null"
},
"__guid": {
"type": "string",
"value": "d952abe5-b7bb-48b7-8205-e87c0b336499"
},
"__version": {
"type": "long",
"value": 1
},
"__type.hive_db.clusterName": {
"type": "string",
"value": "{\"multiplicity\":\"{\\\"lower\\\":1,\\\"upper\\\":1,\\\"isUnique\\\":false}\",\"isIndexable\":true,\"isComposite\":false,\"reverseAttributeName\":null,\"dataType\":\"string\",\"name\":\"clusterName\",\"isUnique\":false}"
},
"__type.category": {
"type": "string",
"value": "CLASS"
},
"__type.version": {
"type": "string",
"value": "1.0"
},
"__type.hive_db.location": {
"type": "string",
"value": "{\"multiplicity\":\"{\\\"lower\\\":0,\\\"upper\\\":1,\\\"isUnique\\\":false}\",\"isIndexable\":false,\"isComposite\":false,\"reverseAttributeName\":null,\"dataType\":\"string\",\"name\":\"location\",\"isUnique\":false}"
},
"__createdBy": {
"type": "string",
"value": "root"
},
"__modificationTimestamp": {
"type": "long",
"value": 1522707676466
},
"__type": {
"type": "string",
"value": "typeSystem"
},
"__type.description": {
"type": "string",
"value": "hive_db"
},
"__timestamp": {
"type": "long",
"value": 1522707676466
},
"__type.hive_db.ownerType": {
"type": "string",
"value": "{\"multiplicity\":\"{\\\"lower\\\":0,\\\"upper\\\":1,\\\"isUnique\\\":false}\",\"isIndexable\":false,\"isComposite\":false,\"reverseAttributeName\":null,\"dataType\":\"hive_principal_type\",\"name\":\"ownerType\",\"isUnique\":false}"
},
"_id": 33080,
"_type": "vertex"
}, {
"__type.name": {
"type": "string",
"value": "Infrastructure"
},
"__version": {
"type": "long",
"value": 1
},
"__type.category": {
"type": "string",
"value": "CLASS"
},
"__modifiedBy": {
"type": "string",
"value": "root"
},
"__type.version": {
"type": "string",
"value": "1.0"
},
"__createdBy": {
"type": "string",
"value": "root"
},
"__modificationTimestamp": {
"type": "long",
"value": 1522707670799
},
"__type": {
"type": "string",
"value": "typeSystem"
},
"__type.options": {
"type": "string",
"value": "null"
},
"__type.description": {
"type": "string",
"value": "Infrastructure"
},
"__guid": {
"type": "string",
"value": "6330bb1f-cefe-4185-b245-b366846f7309"
},
"__timestamp": {
"type": "long",
"value": 1522707670799
},
"_id": 16712,
"_type": "vertex"
}, {
"__type.name": {
"type": "string",
"value": "Asset"
},
"__modifiedBy": {
"type": "string",
"value": "root"
},
"__type.Asset.name": {
"type": "string",
"value": "{\"multiplicity\":\"{\\\"lower\\\":1,\\\"upper\\\":1,\\\"isUnique\\\":false}\",\"isIndexable\":true,\"isComposite\":false,\"reverseAttributeName\":null,\"dataType\":\"string\",\"name\":\"name\",\"isUnique\":false}"
},
"__type.Asset.owner": {
"type": "string",
"value": "{\"multiplicity\":\"{\\\"lower\\\":0,\\\"upper\\\":1,\\\"isUnique\\\":false}\",\"isIndexable\":true,\"isComposite\":false,\"reverseAttributeName\":null,\"dataType\":\"string\",\"name\":\"owner\",\"isUnique\":false}"
},
"__type.options": {
"type": "string",
"value": "null"
},
"__guid": {
"type": "string",
"value": "e4cbc6bb-f271-46d8-bffd-55beac027cca"
},
"__type.Asset": {
"type": "list",
"value": [{
"type": "string",
"value": "name"
}, {
"type": "string",
"value": "description"
}, {
"type": "string",
"value": "owner"
}]
},
"__version": {
"type": "long",
"value": 1
},
"__type.category": {
"type": "string",
"value": "CLASS"
},
"__type.version": {
"type": "string",
"value": "1.0"
},
"__createdBy": {
"type": "string",
"value": "root"
},
"__modificationTimestamp": {
"type": "long",
"value": 1522707670076
},
"__type": {
"type": "string",
"value": "typeSystem"
},
"__type.Asset.description": {
"type": "string",
"value": "{\"multiplicity\":\"{\\\"lower\\\":0,\\\"upper\\\":1,\\\"isUnique\\\":false}\",\"isIndexable\":false,\"isComposite\":false,\"reverseAttributeName\":null,\"dataType\":\"string\",\"name\":\"description\",\"isUnique\":false}"
},
"__type.description": {
"type": "string",
"value": "Asset"
},
"__timestamp": {
"type": "long",
"value": 1522707670076
},
"_id": 16720,
"_type": "vertex"
}, {
"__type.name": {
"type": "string",
"value": "__AtlasUserProfile"
},
"__modifiedBy": {
"type": "string",
"value": "root"
},
"__type.options": {
"type": "string",
"value": "null"
},
"__type.__AtlasUserProfile.name": {
"type": "string",
"value": "{\"multiplicity\":\"{\\\"lower\\\":1,\\\"upper\\\":1,\\\"isUnique\\\":false}\",\"isIndexable\":true,\"isComposite\":false,\"reverseAttributeName\":null,\"dataType\":\"string\",\"name\":\"name\",\"isUnique\":true}"
},
"__guid": {
"type": "string",
"value": "5a4271e0-dd42-4a80-b43e-377abefec3f5"
},
"__type.__AtlasUserProfile.fullName": {
"type": "string",
"value": "{\"multiplicity\":\"{\\\"lower\\\":0,\\\"upper\\\":1,\\\"isUnique\\\":false}\",\"isIndexable\":false,\"isComposite\":false,\"reverseAttributeName\":null,\"dataType\":\"string\",\"name\":\"fullName\",\"isUnique\":false}"
},
"__version": {
"type": "long",
"value": 1
},
"__type.__AtlasUserProfile.savedSearches": {
"type": "string",
"value": "{\"multiplicity\":\"{\\\"lower\\\":0,\\\"upper\\\":2147483647,\\\"isUnique\\\":false}\",\"isIndexable\":false,\"isComposite\":true,\"reverseAttributeName\":null,\"dataType\":\"array<__AtlasUserSavedSearch>\",\"name\":\"savedSearches\",\"isUnique\":false}"
},
"__type.category": {
"type": "string",
"value": "CLASS"
},
"__type.version": {
"type": "string",
"value": "1.0"
},
"__type.__AtlasUserProfile": {
"type": "list",
"value": [{
"type": "string",
"value": "name"
}, {
"type": "string",
"value": "fullName"
}, {
"type": "string",
"value": "savedSearches"
}]
},
"__createdBy": {
"type": "string",
"value": "root"
},
"__modificationTimestamp": {
"type": "long",
"value": 1522707671513
},
"__type": {
"type": "string",
"value": "typeSystem"
},
"__type.description": {
"type": "string",
"value": "__AtlasUserProfile"
},
"__timestamp": {
"type": "long",
"value": 1522707671513
},
"_id": 33104,
"_type": "vertex"
}, {
"__type.name": {
"type": "string",
"value": "__AtlasUserSavedSearch"
},
"__type.__AtlasUserSavedSearch.searchType": {
"type": "string",
"value": "{\"multiplicity\":\"{\\\"lower\\\":1,\\\"upper\\\":1,\\\"isUnique\\\":false}\",\"isIndexable\":true,\"isComposite\":false,\"reverseAttributeName\":null,\"dataType\":\"string\",\"name\":\"searchType\",\"isUnique\":false}"
},
"__modifiedBy": {
"type": "string",
"value": "root"
},
"__type.__AtlasUserSavedSearch.ownerName": {
"type": "string",
"value": "{\"multiplicity\":\"{\\\"lower\\\":1,\\\"upper\\\":1,\\\"isUnique\\\":false}\",\"isIndexable\":false,\"isComposite\":false,\"reverseAttributeName\":null,\"dataType\":\"string\",\"name\":\"ownerName\",\"isUnique\":false}"
},
"__type.__AtlasUserSavedSearch": {
"type": "list",
"value": [{
"type": "string",
"value": "name"
}, {
"type": "string",
"value": "ownerName"
}, {
"type": "string",
"value": "uniqueName"
}, {
"type": "string",
"value": "searchType"
}, {
"type": "string",
"value": "searchParameters"
}]
},
"__type.options": {
"type": "string",
"value": "null"
},
"__type.__AtlasUserSavedSearch.name": {
"type": "string",
"value": "{\"multiplicity\":\"{\\\"lower\\\":1,\\\"upper\\\":1,\\\"isUnique\\\":false}\",\"isIndexable\":false,\"isComposite\":false,\"reverseAttributeName\":null,\"dataType\":\"string\",\"name\":\"name\",\"isUnique\":false}"
},
"__type.__AtlasUserSavedSearch.searchParameters": {
"type": "string",
"value": "{\"multiplicity\":\"{\\\"lower\\\":1,\\\"upper\\\":1,\\\"isUnique\\\":false}\",\"isIndexable\":false,\"isComposite\":false,\"reverseAttributeName\":null,\"dataType\":\"string\",\"name\":\"searchParameters\",\"isUnique\":false}"
},
"__guid": {
"type": "string",
"value": "a8715ea4-573f-4a33-84d1-351706093be1"
},
"__version": {
"type": "long",
"value": 1
},
"__type.__AtlasUserSavedSearch.uniqueName": {
"type": "string",
"value": "{\"multiplicity\":\"{\\\"lower\\\":1,\\\"upper\\\":1,\\\"isUnique\\\":false}\",\"isIndexable\":true,\"isComposite\":false,\"reverseAttributeName\":null,\"dataType\":\"string\",\"name\":\"uniqueName\",\"isUnique\":true}"
},
"__type.category": {
"type": "string",
"value": "CLASS"
},
"__type.version": {
"type": "string",
"value": "1.0"
},
"__createdBy": {
"type": "string",
"value": "root"
},
"__modificationTimestamp": {
"type": "long",
"value": 1522707671534
},
"__type": {
"type": "string",
"value": "typeSystem"
},
"__type.description": {
"type": "string",
"value": "__AtlasUserSavedSearch"
},
"__timestamp": {
"type": "long",
"value": 1522707671534
},
"_id": 49488,
"_type": "vertex"
}, {
"__type.name": {
"type": "string",
"value": "hive_order"
},
"__modifiedBy": {
"type": "string",
"value": "root"
},
"__type.hive_order.order": {
"type": "string",
"value": "{\"multiplicity\":\"{\\\"lower\\\":1,\\\"upper\\\":1,\\\"isUnique\\\":false}\",\"isIndexable\":false,\"isComposite\":false,\"reverseAttributeName\":null,\"dataType\":\"int\",\"name\":\"order\",\"isUnique\":false}"
},
"__type.options": {
"type": "string",
"value": "null"
},
"__guid": {
"type": "string",
"value": "f1941ffb-54b7-402d-b26a-e0e3cd3c4c23"
},
"__version": {
"type": "long",
"value": 1
},
"__type.hive_order": {
"type": "list",
"value": [{
"type": "string",
"value": "order"
}, {
"type": "string",
"value": "col"
}]
},
"__type.hive_order.col": {
"type": "string",
"value": "{\"multiplicity\":\"{\\\"lower\\\":1,\\\"upper\\\":1,\\\"isUnique\\\":false}\",\"isIndexable\":false,\"isComposite\":false,\"reverseAttributeName\":null,\"dataType\":\"string\",\"name\":\"col\",\"isUnique\":false}"
},
"__type.category": {
"type": "string",
"value": "STRUCT"
},
"__type.version": {
"type": "string",
"value": "1.0"
},
"__createdBy": {
"type": "string",
"value": "root"
},
"__modificationTimestamp": {
"type": "long",
"value": 1522707674741
},
"__type": {
"type": "string",
"value": "typeSystem"
},
"__type.description": {
"type": "string",
"value": "hive_order"
},
"__timestamp": {
"type": "long",
"value": 1522707674741
},
"_id": 65872,
"_type": "vertex"
}, {
"__type.hive_storagedesc.outputFormat": {
"type": "string",
"value": "{\"multiplicity\":\"{\\\"lower\\\":0,\\\"upper\\\":1,\\\"isUnique\\\":false}\",\"isIndexable\":false,\"isComposite\":false,\"reverseAttributeName\":null,\"dataType\":\"string\",\"name\":\"outputFormat\",\"isUnique\":false}"
},
"__type.hive_storagedesc.bucketCols": {
"type": "string",
"value": "{\"multiplicity\":\"{\\\"lower\\\":0,\\\"upper\\\":1,\\\"isUnique\\\":false}\",\"isIndexable\":false,\"isComposite\":false,\"reverseAttributeName\":null,\"dataType\":\"array<string>\",\"name\":\"bucketCols\",\"isUnique\":false}"
},
"__type.name": {
"type": "string",
"value": "hive_storagedesc"
},
"__type.hive_storagedesc.parameters": {
"type": "string",
"value": "{\"multiplicity\":\"{\\\"lower\\\":0,\\\"upper\\\":1,\\\"isUnique\\\":false}\",\"isIndexable\":false,\"isComposite\":false,\"reverseAttributeName\":null,\"dataType\":\"map<string,string>\",\"name\":\"parameters\",\"isUnique\":false}"
},
"__modifiedBy": {
"type": "string",
"value": "root"
},
"__type.options": {
"type": "string",
"value": "null"
},
"__type.hive_storagedesc.numBuckets": {
"type": "string",
"value": "{\"multiplicity\":\"{\\\"lower\\\":0,\\\"upper\\\":1,\\\"isUnique\\\":false}\",\"isIndexable\":false,\"isComposite\":false,\"reverseAttributeName\":null,\"dataType\":\"int\",\"name\":\"numBuckets\",\"isUnique\":false}"
},
"__type.hive_storagedesc.sortCols": {
"type": "string",
"value": "{\"multiplicity\":\"{\\\"lower\\\":0,\\\"upper\\\":1,\\\"isUnique\\\":false}\",\"isIndexable\":false,\"isComposite\":false,\"reverseAttributeName\":null,\"dataType\":\"array<hive_order>\",\"name\":\"sortCols\",\"isUnique\":false}"
},
"__guid": {
"type": "string",
"value": "d1f7f74e-20d9-48b6-8b3a-0c70662fe905"
},
"__type.hive_storagedesc": {
"type": "list",
"value": [{
"type": "string",
"value": "table"
}, {
"type": "string",
"value": "location"
}, {
"type": "string",
"value": "inputFormat"
}, {
"type": "string",
"value": "outputFormat"
}, {
"type": "string",
"value": "compressed"
}, {
"type": "string",
"value": "numBuckets"
}, {
"type": "string",
"value": "serdeInfo"
}, {
"type": "string",
"value": "bucketCols"
}, {
"type": "string",
"value": "sortCols"
}, {
"type": "string",
"value": "parameters"
}, {
"type": "string",
"value": "storedAsSubDirectories"
}]
},
"__version": {
"type": "long",
"value": 1
},
"__type.hive_storagedesc.inputFormat": {
"type": "string",
"value": "{\"multiplicity\":\"{\\\"lower\\\":0,\\\"upper\\\":1,\\\"isUnique\\\":false}\",\"isIndexable\":false,\"isComposite\":false,\"reverseAttributeName\":null,\"dataType\":\"string\",\"name\":\"inputFormat\",\"isUnique\":false}"
},
"__type.category": {
"type": "string",
"value": "CLASS"
},
"__type.version": {
"type": "string",
"value": "1.0"
},
"__createdBy": {
"type": "string",
"value": "root"
},
"__type.hive_storagedesc.serdeInfo": {
"type": "string",
"value": "{\"multiplicity\":\"{\\\"lower\\\":0,\\\"upper\\\":1,\\\"isUnique\\\":false}\",\"isIndexable\":false,\"isComposite\":false,\"reverseAttributeName\":null,\"dataType\":\"hive_serde\",\"name\":\"serdeInfo\",\"isUnique\":false}"
},
"__type.hive_storagedesc.storedAsSubDirectories": {
"type": "string",
"value": "{\"multiplicity\":\"{\\\"lower\\\":0,\\\"upper\\\":1,\\\"isUnique\\\":false}\",\"isIndexable\":false,\"isComposite\":false,\"reverseAttributeName\":null,\"dataType\":\"boolean\",\"name\":\"storedAsSubDirectories\",\"isUnique\":false}"
},
"__modificationTimestamp": {
"type": "long",
"value": 1522707676433
},
"__type": {
"type": "string",
"value": "typeSystem"
},
"__type.hive_storagedesc.location": {
"type": "string",
"value": "{\"multiplicity\":\"{\\\"lower\\\":0,\\\"upper\\\":1,\\\"isUnique\\\":false}\",\"isIndexable\":false,\"isComposite\":false,\"reverseAttributeName\":null,\"dataType\":\"string\",\"name\":\"location\",\"isUnique\":false}"
},
"__type.hive_storagedesc.compressed": {
"type": "string",
"value": "{\"multiplicity\":\"{\\\"lower\\\":1,\\\"upper\\\":1,\\\"isUnique\\\":false}\",\"isIndexable\":false,\"isComposite\":false,\"reverseAttributeName\":null,\"dataType\":\"boolean\",\"name\":\"compressed\",\"isUnique\":false}"
},
"__type.description": {
"type": "string",
"value": "hive_storagedesc"
},
"__type.hive_storagedesc.table": {
"type": "string",
"value": "{\"multiplicity\":\"{\\\"lower\\\":0,\\\"upper\\\":1,\\\"isUnique\\\":false}\",\"isIndexable\":false,\"isComposite\":false,\"reverseAttributeName\":\"sd\",\"dataType\":\"hive_table\",\"name\":\"table\",\"isUnique\":false}"
},
"__timestamp": {
"type": "long",
"value": 1522707676433
},
"_id": 82256,
"_type": "vertex"
}, {
"hdfs_path.hashCode": {
"type": "biginteger",
"value": 612361213421234
},
"hdfs_path.numberOfReplicas": {
"type": "integer",
"value": 3
},
"__modifiedBy": {
"type": "string",
"value": "admin"
},
"fs_path.group": {
"type": "string",
"value": "invoices"
},
"Referenceable.qualifiedName": {
"type": "string",
"value": "YTD2018@CL1"
},
"hdfs_path.clusterName": {
"type": "string",
"value": "CL1"
},
"__version": {
"type": "integer",
"value": 0
},
"fs_path.path": {
"type": "string",
"value": "/warehouse/accounts/invoices/YTD2018.xlsx"
},
"fs_path.isSymlink": {
"type": "boolean",
"value": false
},
"hdfs_path.retention": {
"type": "bigdecimal",
"value": 125353
},
"__typeName": {
"type": "string",
"value": "hdfs_path"
},
"fs_path.fileSize": {
"type": "long",
"value": 8899
},
"__timestamp": {
"type": "long",
"value": 1522707838909
},
"Asset.name": {
"type": "string",
"value": "YTD-2018"
},
"__state": {
"type": "string",
"value": "ACTIVE"
},
"Asset.description": {
"type": "string",
"value": "YTD for Year 2018"
},
"hdfs_path.lastChanged": {
"type": "long",
"value": 1523430000000
},
"entityText": {
"type": "string",
"value": "hdfs_path modifiedTime Wed Apr 11 00:00:00 PDT 2018 isFile true numberOfReplicas 3 qualifiedName YTD2018@CL1 lastChanged Wed Apr 11 00:00:00 PDT 2018 description YTD for Year 2018 path /warehouse/accounts/invoices/YTD2018.xlsx createTime Wed Apr 11 00:00:00 PDT 2018 fileSize 8899 hashCode 612361213421234 clusterName CL1 name YTD-2018 isSymlink false retention 125353 group invoices "
},
"fs_path.modifiedTime": {
"type": "long",
"value": 1523430000000
},
"__guid": {
"type": "string",
"value": "a64c2e56-cf2f-4d96-9530-bb6b0c4f4f64"
},
"__superTypeNames": {
"type": "list",
"value": [{
"type": "string",
"value": "Asset"
}, {
"type": "string",
"value": "DataSet"
}, {
"type": "string",
"value": "Referenceable"
}, {
"type": "string",
"value": "fs_path"
}]
},
"fs_path.createTime": {
"type": "long",
"value": 1523430000000
},
"__createdBy": {
"type": "string",
"value": "admin"
},
"__modificationTimestamp": {
"type": "long",
"value": 1522707838909
},
"fs_path.isFile": {
"type": "boolean",
"value": true
},
"_id": 98640,
"_type": "vertex"
}, {
"__type.name": {
"type": "string",
"value": "fs_path"
},
"__modifiedBy": {
"type": "string",
"value": "root"
},
"__type.fs_path": {
"type": "list",
"value": [{
"type": "string",
"value": "path"
}, {
"type": "string",
"value": "createTime"
}, {
"type": "string",
"value": "modifiedTime"
}, {
"type": "string",
"value": "isFile"
}, {
"type": "string",
"value": "isSymlink"
}, {
"type": "string",
"value": "fileSize"
}, {
"type": "string",
"value": "group"
}, {
"type": "string",
"value": "posixPermissions"
}]
},
"__type.options": {
"type": "string",
"value": "null"
},
"__guid": {
"type": "string",
"value": "6de5e3e3-ce22-4160-a019-8b9e3329d5dc"
},
"__type.fs_path.path": {
"type": "string",
"value": "{\"multiplicity\":\"{\\\"lower\\\":1,\\\"upper\\\":1,\\\"isUnique\\\":false}\",\"isIndexable\":true,\"isComposite\":false,\"reverseAttributeName\":null,\"dataType\":\"string\",\"name\":\"path\",\"isUnique\":false}"
},
"__version": {
"type": "long",
"value": 1
},
"__type.fs_path.modifiedTime": {
"type": "string",
"value": "{\"multiplicity\":\"{\\\"lower\\\":0,\\\"upper\\\":1,\\\"isUnique\\\":false}\",\"isIndexable\":true,\"isComposite\":false,\"reverseAttributeName\":null,\"dataType\":\"date\",\"name\":\"modifiedTime\",\"isUnique\":false}"
},
"__type.category": {
"type": "string",
"value": "CLASS"
},
"__type.version": {
"type": "string",
"value": "1.0"
},
"__createdBy": {
"type": "string",
"value": "root"
},
"__type.fs_path.fileSize": {
"type": "string",
"value": "{\"multiplicity\":\"{\\\"lower\\\":0,\\\"upper\\\":1,\\\"isUnique\\\":false}\",\"isIndexable\":false,\"isComposite\":false,\"reverseAttributeName\":null,\"dataType\":\"long\",\"name\":\"fileSize\",\"isUnique\":false}"
},
"__modificationTimestamp": {
"type": "long",
"value": 1522707674164
},
"__type": {
"type": "string",
"value": "typeSystem"
},
"__type.fs_path.isFile": {
"type": "string",
"value": "{\"multiplicity\":\"{\\\"lower\\\":0,\\\"upper\\\":1,\\\"isUnique\\\":false}\",\"isIndexable\":true,\"isComposite\":false,\"reverseAttributeName\":null,\"dataType\":\"boolean\",\"name\":\"isFile\",\"isUnique\":false}"
},
"__type.fs_path.createTime": {
"type": "string",
"value": "{\"multiplicity\":\"{\\\"lower\\\":0,\\\"upper\\\":1,\\\"isUnique\\\":false}\",\"isIndexable\":true,\"isComposite\":false,\"reverseAttributeName\":null,\"dataType\":\"date\",\"name\":\"createTime\",\"isUnique\":false}"
},
"__type.fs_path.isSymlink": {
"type": "string",
"value": "{\"multiplicity\":\"{\\\"lower\\\":0,\\\"upper\\\":1,\\\"isUnique\\\":false}\",\"isIndexable\":false,\"isComposite\":false,\"reverseAttributeName\":null,\"dataType\":\"boolean\",\"name\":\"isSymlink\",\"isUnique\":false}"
},
"__type.fs_path.group": {
"type": "string",
"value": "{\"multiplicity\":\"{\\\"lower\\\":0,\\\"upper\\\":1,\\\"isUnique\\\":false}\",\"isIndexable\":true,\"isComposite\":false,\"reverseAttributeName\":null,\"dataType\":\"string\",\"name\":\"group\",\"isUnique\":false}"
},
"__type.fs_path.posixPermissions": {
"type": "string",
"value": "{\"multiplicity\":\"{\\\"lower\\\":0,\\\"upper\\\":1,\\\"isUnique\\\":false}\",\"isIndexable\":true,\"isComposite\":false,\"reverseAttributeName\":null,\"dataType\":\"fs_permissions\",\"name\":\"posixPermissions\",\"isUnique\":false}"
},
"__type.description": {
"type": "string",
"value": "fs_path"
},
"__timestamp": {
"type": "long",
"value": 1522707674164
},
"_id": 16808,
"_type": "vertex"
}, {
"__type.hdfs_path.extendedAttributes": {
"type": "string",
"value": "{\"multiplicity\":\"{\\\"lower\\\":0,\\\"upper\\\":1,\\\"isUnique\\\":false}\",\"isIndexable\":false,\"isComposite\":false,\"reverseAttributeName\":null,\"dataType\":\"map<string,string>\",\"name\":\"extendedAttributes\",\"isUnique\":false}"
},
"__type.name": {
"type": "string",
"value": "hdfs_path"
},
"__type.hdfs_path.hashCode": {
"type": "string",
"value": "{\"multiplicity\":\"{\\\"lower\\\":0,\\\"upper\\\":1,\\\"isUnique\\\":false}\",\"isIndexable\":false,\"isComposite\":false,\"reverseAttributeName\":null,\"dataType\":\"biginteger\",\"name\":\"hashCode\",\"isUnique\":false}"
},
"__modifiedBy": {
"type": "string",
"value": "root"
},
"__type.hdfs_path.clusterName": {
"type": "string",
"value": "{\"multiplicity\":\"{\\\"lower\\\":0,\\\"upper\\\":1,\\\"isUnique\\\":false}\",\"isIndexable\":true,\"isComposite\":false,\"reverseAttributeName\":null,\"dataType\":\"string\",\"name\":\"clusterName\",\"isUnique\":false}"
},
"__type.options": {
"type": "string",
"value": "null"
},
"__type.hdfs_path": {
"type": "list",
"value": [{
"type": "string",
"value": "clusterName"
}, {
"type": "string",
"value": "numberOfReplicas"
}, {
"type": "string",
"value": "extendedAttributes"
}, {
"type": "string",
"value": "retention"
}, {
"type": "string",
"value": "hashCode"
}, {
"type": "string",
"value": "lastChanged"
}]
},
"__guid": {
"type": "string",
"value": "60ddc14b-de71-433d-bc8b-ae5c5713b1b0"
},
"__version": {
"type": "long",
"value": 1
},
"__type.hdfs_path.retention": {
"type": "string",
"value": "{\"multiplicity\":\"{\\\"lower\\\":0,\\\"upper\\\":1,\\\"isUnique\\\":false}\",\"isIndexable\":false,\"isComposite\":false,\"reverseAttributeName\":null,\"dataType\":\"bigdecimal\",\"name\":\"retention\",\"isUnique\":false}"
},
"__type.category": {
"type": "string",
"value": "CLASS"
},
"__type.version": {
"type": "string",
"value": "1.0"
},
"__createdBy": {
"type": "string",
"value": "root"
},
"__type.hdfs_path.numberOfReplicas": {
"type": "string",
"value": "{\"multiplicity\":\"{\\\"lower\\\":0,\\\"upper\\\":1,\\\"isUnique\\\":false}\",\"isIndexable\":true,\"isComposite\":false,\"reverseAttributeName\":null,\"dataType\":\"int\",\"name\":\"numberOfReplicas\",\"isUnique\":false}"
},
"__type.hdfs_path.lastChanged": {
"type": "string",
"value": "{\"multiplicity\":\"{\\\"lower\\\":0,\\\"upper\\\":1,\\\"isUnique\\\":false}\",\"isIndexable\":false,\"isComposite\":false,\"reverseAttributeName\":null,\"dataType\":\"date\",\"name\":\"lastChanged\",\"isUnique\":false}"
},
"__modificationTimestamp": {
"type": "long",
"value": 1522707674212
},
"__type": {
"type": "string",
"value": "typeSystem"
},
"__type.description": {
"type": "string",
"value": "hdfs_path"
},
"__timestamp": {
"type": "long",
"value": 1522707674212
},
"_id": 33192,
"_type": "vertex"
}, {
"__type.name": {
"type": "string",
"value": "hive_column"
},
"__type.hive_column.table": {
"type": "string",
"value": "{\"multiplicity\":\"{\\\"lower\\\":0,\\\"upper\\\":1,\\\"isUnique\\\":false}\",\"isIndexable\":false,\"isComposite\":false,\"reverseAttributeName\":null,\"dataType\":\"hive_table\",\"name\":\"table\",\"isUnique\":false}"
},
"__modifiedBy": {
"type": "string",
"value": "root"
},
"__type.options": {
"type": "string",
"value": "{\"schemaAttributes\":\"[\\\"name\\\", \\\"description\\\", \\\"owner\\\", \\\"type\\\", \\\"comment\\\", \\\"position\\\"]\"}"
},
"__type.hive_column": {
"type": "list",
"value": [{
"type": "string",
"value": "type"
}, {
"type": "string",
"value": "comment"
}, {
"type": "string",
"value": "position"
}, {
"type": "string",
"value": "table"
}]
},
"__type.hive_column.type": {
"type": "string",
"value": "{\"multiplicity\":\"{\\\"lower\\\":1,\\\"upper\\\":1,\\\"isUnique\\\":false}\",\"isIndexable\":true,\"isComposite\":false,\"reverseAttributeName\":null,\"dataType\":\"string\",\"name\":\"type\",\"isUnique\":false}"
},
"__type.hive_column.comment": {
"type": "string",
"value": "{\"multiplicity\":\"{\\\"lower\\\":0,\\\"upper\\\":1,\\\"isUnique\\\":false}\",\"isIndexable\":false,\"isComposite\":false,\"reverseAttributeName\":null,\"dataType\":\"string\",\"name\":\"comment\",\"isUnique\":false}"
},
"__guid": {
"type": "string",
"value": "9aaf6283-44ee-4f96-83f8-050faff83226"
},
"__version": {
"type": "long",
"value": 4
},
"__type.category": {
"type": "string",
"value": "CLASS"
},
"__type.version": {
"type": "string",
"value": "1.3"
},
"__createdBy": {
"type": "string",
"value": "root"
},
"__modificationTimestamp": {
"type": "long",
"value": 1522707677531
},
"__type": {
"type": "string",
"value": "typeSystem"
},
"__type.hive_column.position": {
"type": "string",
"value": "{\"multiplicity\":\"{\\\"lower\\\":0,\\\"upper\\\":1,\\\"isUnique\\\":false}\",\"isIndexable\":false,\"isComposite\":false,\"reverseAttributeName\":null,\"dataType\":\"int\",\"name\":\"position\",\"isUnique\":false}"
},
"__type.description": {
"type": "string",
"value": "hive_column"
},
"__timestamp": {
"type": "long",
"value": 1522707676478
},
"_id": 49576,
"_type": "vertex"
}, {
"__type.name": {
"type": "string",
"value": "Referenceable"
},
"__modifiedBy": {
"type": "string",
"value": "root"
},
"__type.options": {
"type": "string",
"value": "null"
},
"__type.Referenceable": {
"type": "list",
"value": [{
"type": "string",
"value": "qualifiedName"
}]
},
"__guid": {
"type": "string",
"value": "79d0ca22-fe92-4904-8fd4-5581d9f1c0f8"
},
"__version": {
"type": "long",
"value": 1
},
"__type.category": {
"type": "string",
"value": "CLASS"
},
"__type.version": {
"type": "string",
"value": "1.0"
},
"__createdBy": {
"type": "string",
"value": "root"
},
"__modificationTimestamp": {
"type": "long",
"value": 1522707668702
},
"__type": {
"type": "string",
"value": "typeSystem"
},
"__type.Referenceable.qualifiedName": {
"type": "string",
"value": "{\"multiplicity\":\"{\\\"lower\\\":1,\\\"upper\\\":1,\\\"isUnique\\\":false}\",\"isIndexable\":true,\"isComposite\":false,\"reverseAttributeName\":null,\"dataType\":\"string\",\"name\":\"qualifiedName\",\"isUnique\":true}"
},
"__type.description": {
"type": "string",
"value": "Referenceable"
},
"__timestamp": {
"type": "long",
"value": 1522707668702
},
"_id": 16824,
"_type": "vertex"
}, {
"__type.name": {
"type": "string",
"value": "DataSet"
},
"__version": {
"type": "long",
"value": 1
},
"__type.category": {
"type": "string",
"value": "CLASS"
},
"__modifiedBy": {
"type": "string",
"value": "root"
},
"__type.version": {
"type": "string",
"value": "1.0"
},
"__createdBy": {
"type": "string",
"value": "root"
},
"__modificationTimestamp": {
"type": "long",
"value": 1522707670107
},
"__type": {
"type": "string",
"value": "typeSystem"
},
"__type.options": {
"type": "string",
"value": "null"
},
"__type.description": {
"type": "string",
"value": "DataSet"
},
"__guid": {
"type": "string",
"value": "5bda2ac5-4d9f-4b34-8a8f-0f40d29ceaad"
},
"__timestamp": {
"type": "long",
"value": 1522707670107
},
"_id": 33208,
"_type": "vertex"
}],
"edges": [{
"_id": "22a5-pd4-2711-cs0",
"_type": "edge",
"_outV": 32872,
"_inV": 16560,
"_label": "__type..supertype"
}, {
"_id": "1el0-peo-4vf9-cq0",
"_type": "edge",
"_outV": 32928,
"_inV": 16488,
"_label": "__type.edge.fs_permissions.group"
}, {
"_id": "1g5w-peo-4x05-cq0",
"_type": "edge",
"_outV": 32928,
"_inV": 16488,
"_label": "__type.edge.fs_permissions.user"
}, {
"_id": "1hqs-peo-4yl1-cq0",
"_type": "edge",
"_outV": 32928,
"_inV": 16488,
"_label": "__type.edge.fs_permissions.others"
}, {
"_id": "vme-cs0-2711-cwg",
"_type": "edge",
"_outV": 16560,
"_inV": 16720,
"_label": "__type..supertype"
}, {
"_id": "u1i-cs0-2711-czc",
"_type": "edge",
"_outV": 16560,
"_inV": 16824,
"_label": "__type..supertype"
}, {
"_id": "qvq-cs0-28lx-pmg",
"_type": "edge",
"_outV": 16560,
"_inV": 33208,
"_label": "__type.edge.Process.inputs"
}, {
"_id": "sgm-cs0-2a6t-pmg",
"_type": "edge",
"_outV": 16560,
"_inV": 33208,
"_label": "__type.edge.Process.outputs"
}, {
"_id": "1pna-pf4-2711-cs0",
"_type": "edge",
"_outV": 32944,
"_inV": 16560,
"_label": "__type..supertype"
}, {
"_id": "1o2e-pf4-9tzp-pd4",
"_type": "edge",
"_outV": 32944,
"_inV": 32872,
"_label": "__type.edge.hive_column_lineage.query"
}, {
"_id": "28mf-cvs-2711-pmg",
"_type": "edge",
"_outV": 16696,
"_inV": 33208,
"_label": "__type..supertype"
}, {
"_id": "22av-cvs-9frp-piw",
"_type": "edge",
"_outV": 16696,
"_inV": 33080,
"_label": "__type.edge.hive_table.db"
}, {
"_id": "23vr-cvs-9hcl-1rgw",
"_type": "edge",
"_outV": 16696,
"_inV": 82256,
"_label": "__type.edge.hive_table.sd"
}, {
"_id": "25gn-cvs-9ixh-1294",
"_type": "edge",
"_outV": 16696,
"_inV": 49576,
"_label": "__type.edge.hive_table.partitionKeys"
}, {
"_id": "271j-cvs-9kid-1294",
"_type": "edge",
"_outV": 16696,
"_inV": 49576,
"_label": "__type.edge.hive_table.columns"
}, {
"_id": "2dd3-piw-2711-cwg",
"_type": "edge",
"_outV": 33080,
"_inV": 16720,
"_label": "__type..supertype"
}, {
"_id": "2bs7-piw-2711-czc",
"_type": "edge",
"_outV": 33080,
"_inV": 16824,
"_label": "__type..supertype"
}, {
"_id": "2a7b-piw-9qtx-121s",
"_type": "edge",
"_outV": 33080,
"_inV": 49312,
"_label": "__type.edge.hive_db.ownerType"
}, {
"_id": "nqh-cw8-2711-cwg",
"_type": "edge",
"_outV": 16712,
"_inV": 16720,
"_label": "__type..supertype"
}, {
"_id": "m5l-cw8-2711-czc",
"_type": "edge",
"_outV": 16712,
"_inV": 16824,
"_label": "__type..supertype"
}, {
"_id": "2ey2-pjk-2711-crk",
"_type": "edge",
"_outV": 33104,
"_inV": 16544,
"_label": "__type..supertype"
}, {
"_id": "2dd6-pjk-2brp-126o",
"_type": "edge",
"_outV": 33104,
"_inV": 49488,
"_label": "__type.edge.__AtlasUserProfile.savedSearches"
}, {
"_id": "2giy-126o-2711-crk",
"_type": "edge",
"_outV": 49488,
"_inV": 16544,
"_label": "__type..supertype"
}, {
"_id": "4fmy-1rgw-2711-czc",
"_type": "edge",
"_outV": 82256,
"_inV": 16824,
"_label": "__type..supertype"
}, {
"_id": "4awa-1rgw-9m39-cvs",
"_type": "edge",
"_outV": 82256,
"_inV": 16696,
"_label": "__type.edge.hive_storagedesc.table"
}, {
"_id": "4ch6-1rgw-9no5-cqw",
"_type": "edge",
"_outV": 82256,
"_inV": 16520,
"_label": "__type.edge.hive_storagedesc.serdeInfo"
}, {
"_id": "4e22-1rgw-9p91-1ets",
"_type": "edge",
"_outV": 82256,
"_inV": 65872,
"_label": "__type.edge.hive_storagedesc.sortCols"
}, {
"_id": "1xkl-cyw-2711-pmg",
"_type": "edge",
"_outV": 16808,
"_inV": 33208,
"_label": "__type..supertype"
}, {
"_id": "1vzp-cyw-505x-peo",
"_type": "edge",
"_outV": 16808,
"_inV": 32928,
"_label": "__type.edge.fs_path.posixPermissions"
}, {
"_id": "1z5h-pm0-2711-cyw",
"_type": "edge",
"_outV": 33192,
"_inV": 16808,
"_label": "__type..supertype"
}, {
"_id": "2t6d-1294-2711-pmg",
"_type": "edge",
"_outV": 49576,
"_inV": 33208,
"_label": "__type..supertype"
}, {
"_id": "2rlh-1294-9set-cvs",
"_type": "edge",
"_outV": 49576,
"_inV": 16696,
"_label": "__type.edge.hive_column.table"
}, {
"_id": "1bg7-pmg-2711-cwg",
"_type": "edge",
"_outV": 33208,
"_inV": 16720,
"_label": "__type..supertype"
}, {
"_id": "19vb-pmg-2711-czc",
"_type": "edge",
"_outV": 33208,
"_inV": 16824,
"_label": "__type..supertype"
}],
"vertexCount": 23,
"edgeCount": 34
}
\ No newline at end of file
{"enumDefs":[{"category":"ENUM","guid":"560cdc45-5883-448f-9217-eafdaf577bf5","createdBy":"root","updatedBy":"root","createTime":1522707674715,"updateTime":1522707674715,"version":1,"name":"hive_principal_type","description":"hive_principal_type","typeVersion":"1.0","elementDefs":[{"value":"USER","ordinal":1},{"value":"ROLE","ordinal":2},{"value":"GROUP","ordinal":3}]},{"category":"ENUM","guid":"ee607a83-e4ac-4a93-8bdb-cbae7b89c056","createdBy":"root","updatedBy":"root","createTime":1522707673425,"updateTime":1522707673425,"version":1,"name":"file_action","description":"file_action","typeVersion":"1.0","elementDefs":[{"value":"NONE","ordinal":0},{"value":"EXECUTE","ordinal":1},{"value":"WRITE","ordinal":2},{"value":"WRITE_EXECUTE","ordinal":3},{"value":"READ","ordinal":4},{"value":"READ_EXECUTE","ordinal":5},{"value":"READ_WRITE","ordinal":6},{"value":"ALL","ordinal":7}]}],"structDefs":[{"category":"STRUCT","guid":"29e2774b-fc9f-44f1-adf9-f7bdc7db1a70","createdBy":"root","updatedBy":"root","createTime":1522707675680,"updateTime":1522707675680,"version":1,"name":"hive_serde","description":"hive_serde","typeVersion":"1.0","attributeDefs":[{"name":"name","typeName":"string","isOptional":true,"cardinality":"SINGLE","valuesMinCount":0,"valuesMaxCount":1,"isUnique":false,"isIndexable":false},{"name":"serializationLib","typeName":"string","isOptional":true,"cardinality":"SINGLE","valuesMinCount":0,"valuesMaxCount":1,"isUnique":false,"isIndexable":false},{"name":"parameters","typeName":"map<string,string>","isOptional":true,"cardinality":"SINGLE","valuesMinCount":0,"valuesMaxCount":1,"isUnique":false,"isIndexable":false}]},{"category":"STRUCT","guid":"f1941ffb-54b7-402d-b26a-e0e3cd3c4c23","createdBy":"root","updatedBy":"root","createTime":1522707674741,"updateTime":1522707674741,"version":1,"name":"hive_order","description":"hive_order","typeVersion":"1.0","attributeDefs":[{"name":"order","typeName":"int","isOptional":false,"cardinality":"SINGLE","valuesMinCount":1,"valuesMaxCount":1,"isUnique":false,"isIndexable":false},{"name":"col","typeName":"string","isOptional":false,"cardinality":"SINGLE","valuesMinCount":1,"valuesMaxCount":1,"isUnique":false,"isIndexable":false}]},{"category":"STRUCT","guid":"24f03770-fb23-4efe-98c0-65ed26816237","createdBy":"root","updatedBy":"root","createTime":1522707673475,"updateTime":1522707673475,"version":1,"name":"fs_permissions","description":"fs_permissions","typeVersion":"1.0","attributeDefs":[{"name":"group","typeName":"file_action","isOptional":false,"cardinality":"SINGLE","valuesMinCount":1,"valuesMaxCount":1,"isUnique":false,"isIndexable":true},{"name":"user","typeName":"file_action","isOptional":false,"cardinality":"SINGLE","valuesMinCount":1,"valuesMaxCount":1,"isUnique":false,"isIndexable":true},{"name":"sticky","typeName":"boolean","isOptional":false,"cardinality":"SINGLE","valuesMinCount":1,"valuesMaxCount":1,"isUnique":false,"isIndexable":true},{"name":"others","typeName":"file_action","isOptional":false,"cardinality":"SINGLE","valuesMinCount":1,"valuesMaxCount":1,"isUnique":false,"isIndexable":true}]}],"classificationDefs":[{"category":"CLASSIFICATION","guid":"1732545c-9e0e-45ec-95aa-09c04f0b2a63","createdBy":"root","updatedBy":"root","createTime":1522707667929,"updateTime":1522707667929,"version":1,"name":"TaxonomyTerm","description":"TaxonomyTerm","typeVersion":"1.0","attributeDefs":[{"name":"atlas.taxonomy","typeName":"string","isOptional":true,"cardinality":"SINGLE","valuesMinCount":0,"valuesMaxCount":1,"isUnique":false,"isIndexable":false}],"superTypes":[]}],"entityDefs":[{"category":"ENTITY","guid":"5bda2ac5-4d9f-4b34-8a8f-0f40d29ceaad","createdBy":"root","updatedBy":"root","createTime":1522707670107,"updateTime":1522707670107,"version":1,"name":"DataSet","description":"DataSet","typeVersion":"1.0","attributeDefs":[],"superTypes":["Referenceable","Asset"]},{"category":"ENTITY","guid":"a8715ea4-573f-4a33-84d1-351706093be1","createdBy":"root","updatedBy":"root","createTime":1522707671534,"updateTime":1522707671534,"version":1,"name":"__AtlasUserSavedSearch","description":"__AtlasUserSavedSearch","typeVersion":"1.0","attributeDefs":[{"name":"name","typeName":"string","isOptional":false,"cardinality":"SINGLE","valuesMinCount":1,"valuesMaxCount":1,"isUnique":false,"isIndexable":false},{"name":"ownerName","typeName":"string","isOptional":false,"cardinality":"SINGLE","valuesMinCount":1,"valuesMaxCount":1,"isUnique":false,"isIndexable":false},{"name":"uniqueName","typeName":"string","isOptional":false,"cardinality":"SINGLE","valuesMinCount":1,"valuesMaxCount":1,"isUnique":true,"isIndexable":true},{"name":"searchType","typeName":"string","isOptional":false,"cardinality":"SINGLE","valuesMinCount":1,"valuesMaxCount":1,"isUnique":false,"isIndexable":true},{"name":"searchParameters","typeName":"string","isOptional":false,"cardinality":"SINGLE","valuesMinCount":1,"valuesMaxCount":1,"isUnique":false,"isIndexable":false}],"superTypes":["__internal"]},{"category":"ENTITY","guid":"89d95a88-a996-45cc-a1c0-2536bd99a68c","createdBy":"root","updatedBy":"root","createTime":1522707671495,"updateTime":1522707671495,"version":1,"name":"Process","description":"Process","typeVersion":"1.0","attributeDefs":[{"name":"inputs","typeName":"array<DataSet>","isOptional":true,"cardinality":"SINGLE","valuesMinCount":0,"valuesMaxCount":1,"isUnique":false,"isIndexable":false},{"name":"outputs","typeName":"array<DataSet>","isOptional":true,"cardinality":"SINGLE","valuesMinCount":0,"valuesMaxCount":1,"isUnique":false,"isIndexable":false}],"superTypes":["Referenceable","Asset"]},{"category":"ENTITY","guid":"2233745d-f770-4e13-8fd7-75ea5254f221","createdBy":"root","updatedBy":"root","createTime":1522707676395,"updateTime":1522707677388,"version":2,"name":"hive_table","description":"hive_table","typeVersion":"1.1","options":{"schemaElementsAttribute":"columns"},"attributeDefs":[{"name":"db","typeName":"hive_db","isOptional":false,"cardinality":"SINGLE","valuesMinCount":1,"valuesMaxCount":1,"isUnique":false,"isIndexable":false},{"name":"createTime","typeName":"date","isOptional":true,"cardinality":"SINGLE","valuesMinCount":0,"valuesMaxCount":1,"isUnique":false,"isIndexable":false},{"name":"lastAccessTime","typeName":"date","isOptional":true,"cardinality":"SINGLE","valuesMinCount":0,"valuesMaxCount":1,"isUnique":false,"isIndexable":false},{"name":"comment","typeName":"string","isOptional":true,"cardinality":"SINGLE","valuesMinCount":0,"valuesMaxCount":1,"isUnique":false,"isIndexable":false},{"name":"retention","typeName":"int","isOptional":true,"cardinality":"SINGLE","valuesMinCount":0,"valuesMaxCount":1,"isUnique":false,"isIndexable":false},{"name":"sd","typeName":"hive_storagedesc","isOptional":true,"cardinality":"SINGLE","valuesMinCount":0,"valuesMaxCount":1,"isUnique":false,"isIndexable":false,"constraints":[{"type":"ownedRef"}]},{"name":"partitionKeys","typeName":"array<hive_column>","isOptional":true,"cardinality":"SINGLE","valuesMinCount":0,"valuesMaxCount":1,"isUnique":false,"isIndexable":false,"constraints":[{"type":"ownedRef"}]},{"name":"aliases","typeName":"array<string>","isOptional":true,"cardinality":"SINGLE","valuesMinCount":0,"valuesMaxCount":1,"isUnique":false,"isIndexable":false},{"name":"columns","typeName":"array<hive_column>","isOptional":true,"cardinality":"SINGLE","valuesMinCount":0,"valuesMaxCount":1,"isUnique":false,"isIndexable":false,"constraints":[{"type":"ownedRef"}]},{"name":"parameters","typeName":"map<string,string>","isOptional":true,"cardinality":"SINGLE","valuesMinCount":0,"valuesMaxCount":1,"isUnique":false,"isIndexable":false},{"name":"viewOriginalText","typeName":"string","isOptional":true,"cardinality":"SINGLE","valuesMinCount":0,"valuesMaxCount":1,"isUnique":false,"isIndexable":false},{"name":"viewExpandedText","typeName":"string","isOptional":true,"cardinality":"SINGLE","valuesMinCount":0,"valuesMaxCount":1,"isUnique":false,"isIndexable":false},{"name":"tableType","typeName":"string","isOptional":true,"cardinality":"SINGLE","valuesMinCount":0,"valuesMaxCount":1,"isUnique":false,"isIndexable":false},{"name":"temporary","typeName":"boolean","isOptional":true,"cardinality":"SINGLE","valuesMinCount":0,"valuesMaxCount":1,"isUnique":false,"isIndexable":true}],"superTypes":["DataSet"]},{"category":"ENTITY","guid":"5a4271e0-dd42-4a80-b43e-377abefec3f5","createdBy":"root","updatedBy":"root","createTime":1522707671513,"updateTime":1522707671513,"version":1,"name":"__AtlasUserProfile","description":"__AtlasUserProfile","typeVersion":"1.0","attributeDefs":[{"name":"name","typeName":"string","isOptional":false,"cardinality":"SINGLE","valuesMinCount":1,"valuesMaxCount":1,"isUnique":true,"isIndexable":true},{"name":"fullName","typeName":"string","isOptional":true,"cardinality":"SINGLE","valuesMinCount":0,"valuesMaxCount":1,"isUnique":false,"isIndexable":false},{"name":"savedSearches","typeName":"array<__AtlasUserSavedSearch>","isOptional":true,"cardinality":"LIST","valuesMinCount":0,"valuesMaxCount":2147483647,"isUnique":false,"isIndexable":false,"constraints":[{"type":"ownedRef"}]}],"superTypes":["__internal"]},{"category":"ENTITY","guid":"d952abe5-b7bb-48b7-8205-e87c0b336499","createdBy":"root","updatedBy":"root","createTime":1522707676466,"updateTime":1522707676466,"version":1,"name":"hive_db","description":"hive_db","typeVersion":"1.0","attributeDefs":[{"name":"clusterName","typeName":"string","isOptional":false,"cardinality":"SINGLE","valuesMinCount":1,"valuesMaxCount":1,"isUnique":false,"isIndexable":true},{"name":"location","typeName":"string","isOptional":true,"cardinality":"SINGLE","valuesMinCount":0,"valuesMaxCount":1,"isUnique":false,"isIndexable":false},{"name":"parameters","typeName":"map<string,string>","isOptional":true,"cardinality":"SINGLE","valuesMinCount":0,"valuesMaxCount":1,"isUnique":false,"isIndexable":false},{"name":"ownerType","typeName":"hive_principal_type","isOptional":true,"cardinality":"SINGLE","valuesMinCount":0,"valuesMaxCount":1,"isUnique":false,"isIndexable":false}],"superTypes":["Referenceable","Asset"]},{"category":"ENTITY","guid":"51d50b18-31fe-49b3-ba53-cfbcef964241","createdBy":"root","updatedBy":"root","createTime":1522707675692,"updateTime":1522707675692,"version":1,"name":"hive_process","description":"hive_process","typeVersion":"1.0","attributeDefs":[{"name":"startTime","typeName":"date","isOptional":false,"cardinality":"SINGLE","valuesMinCount":1,"valuesMaxCount":1,"isUnique":false,"isIndexable":false},{"name":"endTime","typeName":"date","isOptional":false,"cardinality":"SINGLE","valuesMinCount":1,"valuesMaxCount":1,"isUnique":false,"isIndexable":false},{"name":"userName","typeName":"string","isOptional":false,"cardinality":"SINGLE","valuesMinCount":1,"valuesMaxCount":1,"isUnique":false,"isIndexable":true},{"name":"operationType","typeName":"string","isOptional":false,"cardinality":"SINGLE","valuesMinCount":1,"valuesMaxCount":1,"isUnique":false,"isIndexable":true},{"name":"queryText","typeName":"string","isOptional":false,"cardinality":"SINGLE","valuesMinCount":1,"valuesMaxCount":1,"isUnique":false,"isIndexable":false},{"name":"queryPlan","typeName":"string","isOptional":false,"cardinality":"SINGLE","valuesMinCount":1,"valuesMaxCount":1,"isUnique":false,"isIndexable":false},{"name":"queryId","typeName":"string","isOptional":false,"cardinality":"SINGLE","valuesMinCount":1,"valuesMaxCount":1,"isUnique":false,"isIndexable":false},{"name":"recentQueries","typeName":"array<string>","isOptional":true,"cardinality":"SINGLE","valuesMinCount":0,"valuesMaxCount":1,"isUnique":false,"isIndexable":false},{"name":"clusterName","typeName":"string","isOptional":true,"cardinality":"SINGLE","valuesMinCount":0,"valuesMaxCount":1,"isUnique":false,"isIndexable":false},{"name":"queryGraph","typeName":"string","isOptional":true,"cardinality":"SINGLE","valuesMinCount":0,"valuesMaxCount":1,"isUnique":false,"isIndexable":false}],"superTypes":["Process"]},{"category":"ENTITY","guid":"d1f7f74e-20d9-48b6-8b3a-0c70662fe905","createdBy":"root","updatedBy":"root","createTime":1522707676433,"updateTime":1522707676433,"version":1,"name":"hive_storagedesc","description":"hive_storagedesc","typeVersion":"1.0","attributeDefs":[{"name":"table","typeName":"hive_table","isOptional":true,"cardinality":"SINGLE","valuesMinCount":0,"valuesMaxCount":1,"isUnique":false,"isIndexable":false,"constraints":[{"type":"inverseRef","params":{"attribute":"sd"}}]},{"name":"location","typeName":"string","isOptional":true,"cardinality":"SINGLE","valuesMinCount":0,"valuesMaxCount":1,"isUnique":false,"isIndexable":false},{"name":"inputFormat","typeName":"string","isOptional":true,"cardinality":"SINGLE","valuesMinCount":0,"valuesMaxCount":1,"isUnique":false,"isIndexable":false},{"name":"outputFormat","typeName":"string","isOptional":true,"cardinality":"SINGLE","valuesMinCount":0,"valuesMaxCount":1,"isUnique":false,"isIndexable":false},{"name":"compressed","typeName":"boolean","isOptional":false,"cardinality":"SINGLE","valuesMinCount":1,"valuesMaxCount":1,"isUnique":false,"isIndexable":false},{"name":"numBuckets","typeName":"int","isOptional":true,"cardinality":"SINGLE","valuesMinCount":0,"valuesMaxCount":1,"isUnique":false,"isIndexable":false},{"name":"serdeInfo","typeName":"hive_serde","isOptional":true,"cardinality":"SINGLE","valuesMinCount":0,"valuesMaxCount":1,"isUnique":false,"isIndexable":false},{"name":"bucketCols","typeName":"array<string>","isOptional":true,"cardinality":"SINGLE","valuesMinCount":0,"valuesMaxCount":1,"isUnique":false,"isIndexable":false},{"name":"sortCols","typeName":"array<hive_order>","isOptional":true,"cardinality":"SINGLE","valuesMinCount":0,"valuesMaxCount":1,"isUnique":false,"isIndexable":false},{"name":"parameters","typeName":"map<string,string>","isOptional":true,"cardinality":"SINGLE","valuesMinCount":0,"valuesMaxCount":1,"isUnique":false,"isIndexable":false},{"name":"storedAsSubDirectories","typeName":"boolean","isOptional":true,"cardinality":"SINGLE","valuesMinCount":0,"valuesMaxCount":1,"isUnique":false,"isIndexable":false}],"superTypes":["Referenceable"]},{"category":"ENTITY","guid":"60ddc14b-de71-433d-bc8b-ae5c5713b1b0","createdBy":"root","updatedBy":"root","createTime":1522707674212,"updateTime":1522707674212,"version":1,"name":"hdfs_path","description":"hdfs_path","typeVersion":"1.0","attributeDefs":[{"name":"clusterName","typeName":"string","isOptional":true,"cardinality":"SINGLE","valuesMinCount":0,"valuesMaxCount":1,"isUnique":false,"isIndexable":true},{"name":"numberOfReplicas","typeName":"int","isOptional":true,"cardinality":"SINGLE","valuesMinCount":0,"valuesMaxCount":1,"isUnique":false,"isIndexable":true},{"name":"extendedAttributes","typeName":"map<string,string>","isOptional":true,"cardinality":"SINGLE","valuesMinCount":0,"valuesMaxCount":1,"isUnique":false,"isIndexable":false},{"name":"retention","typeName":"bigdecimal","isOptional":true,"cardinality":"SINGLE","valuesMinCount":0,"valuesMaxCount":1,"isUnique":false,"isIndexable":false},{"name":"hashCode","typeName":"biginteger","isOptional":true,"cardinality":"SINGLE","valuesMinCount":0,"valuesMaxCount":1,"isUnique":false,"isIndexable":false},{"name":"lastChanged","typeName":"date","isOptional":true,"cardinality":"SINGLE","valuesMinCount":0,"valuesMaxCount":1,"isUnique":false,"isIndexable":false}],"superTypes":["fs_path"]},{"category":"ENTITY","guid":"79d0ca22-fe92-4904-8fd4-5581d9f1c0f8","createdBy":"root","updatedBy":"root","createTime":1522707668702,"updateTime":1522707668702,"version":1,"name":"Referenceable","description":"Referenceable","typeVersion":"1.0","attributeDefs":[{"name":"qualifiedName","typeName":"string","isOptional":false,"cardinality":"SINGLE","valuesMinCount":1,"valuesMaxCount":1,"isUnique":true,"isIndexable":true}],"superTypes":[]},{"category":"ENTITY","guid":"688cdf4c-c187-4ed7-8236-e9067005ab5d","createdBy":"root","updatedBy":"root","createTime":1522707676486,"updateTime":1522707676486,"version":1,"name":"hive_column_lineage","description":"hive_column_lineage","typeVersion":"1.0","attributeDefs":[{"name":"query","typeName":"hive_process","isOptional":false,"cardinality":"SINGLE","valuesMinCount":1,"valuesMaxCount":1,"isUnique":false,"isIndexable":false},{"name":"depenendencyType","typeName":"string","isOptional":false,"cardinality":"SINGLE","valuesMinCount":1,"valuesMaxCount":1,"isUnique":false,"isIndexable":false},{"name":"expression","typeName":"string","isOptional":true,"cardinality":"SINGLE","valuesMinCount":0,"valuesMaxCount":1,"isUnique":false,"isIndexable":false}],"superTypes":["Process"]},{"category":"ENTITY","guid":"6330bb1f-cefe-4185-b245-b366846f7309","createdBy":"root","updatedBy":"root","createTime":1522707670799,"updateTime":1522707670799,"version":1,"name":"Infrastructure","description":"Infrastructure","typeVersion":"1.0","attributeDefs":[],"superTypes":["Referenceable","Asset"]},{"category":"ENTITY","guid":"358d2e1d-7a63-496f-82fb-8327376ba2de","createdBy":"root","updatedBy":"root","createTime":1522707669396,"updateTime":1522707669396,"version":1,"name":"__internal","description":"__internal","typeVersion":"1.0","attributeDefs":[],"superTypes":[]},{"category":"ENTITY","guid":"e4cbc6bb-f271-46d8-bffd-55beac027cca","createdBy":"root","updatedBy":"root","createTime":1522707670076,"updateTime":1522707670076,"version":1,"name":"Asset","description":"Asset","typeVersion":"1.0","attributeDefs":[{"name":"name","typeName":"string","isOptional":false,"cardinality":"SINGLE","valuesMinCount":1,"valuesMaxCount":1,"isUnique":false,"isIndexable":true},{"name":"description","typeName":"string","isOptional":true,"cardinality":"SINGLE","valuesMinCount":0,"valuesMaxCount":1,"isUnique":false,"isIndexable":false},{"name":"owner","typeName":"string","isOptional":true,"cardinality":"SINGLE","valuesMinCount":0,"valuesMaxCount":1,"isUnique":false,"isIndexable":true}],"superTypes":[]},{"category":"ENTITY","guid":"9aaf6283-44ee-4f96-83f8-050faff83226","createdBy":"root","updatedBy":"root","createTime":1522707676478,"updateTime":1522707677531,"version":4,"name":"hive_column","description":"hive_column","typeVersion":"1.3","options":{"schemaAttributes":"[\"name\", \"description\", \"owner\", \"type\", \"comment\", \"position\"]"},"attributeDefs":[{"name":"type","typeName":"string","isOptional":false,"cardinality":"SINGLE","valuesMinCount":1,"valuesMaxCount":1,"isUnique":false,"isIndexable":true},{"name":"comment","typeName":"string","isOptional":true,"cardinality":"SINGLE","valuesMinCount":0,"valuesMaxCount":1,"isUnique":false,"isIndexable":false},{"name":"position","typeName":"int","isOptional":true,"cardinality":"SINGLE","valuesMinCount":0,"valuesMaxCount":1,"isUnique":false,"isIndexable":false},{"name":"table","typeName":"hive_table","isOptional":true,"cardinality":"SINGLE","valuesMinCount":0,"valuesMaxCount":1,"isUnique":false,"isIndexable":false}],"superTypes":["DataSet"]},{"category":"ENTITY","guid":"6de5e3e3-ce22-4160-a019-8b9e3329d5dc","createdBy":"root","updatedBy":"root","createTime":1522707674164,"updateTime":1522707674164,"version":1,"name":"fs_path","description":"fs_path","typeVersion":"1.0","attributeDefs":[{"name":"path","typeName":"string","isOptional":false,"cardinality":"SINGLE","valuesMinCount":1,"valuesMaxCount":1,"isUnique":false,"isIndexable":true},{"name":"createTime","typeName":"date","isOptional":true,"cardinality":"SINGLE","valuesMinCount":0,"valuesMaxCount":1,"isUnique":false,"isIndexable":true},{"name":"modifiedTime","typeName":"date","isOptional":true,"cardinality":"SINGLE","valuesMinCount":0,"valuesMaxCount":1,"isUnique":false,"isIndexable":true},{"name":"isFile","typeName":"boolean","isOptional":true,"cardinality":"SINGLE","valuesMinCount":0,"valuesMaxCount":1,"isUnique":false,"isIndexable":true},{"name":"isSymlink","typeName":"boolean","isOptional":true,"cardinality":"SINGLE","valuesMinCount":0,"valuesMaxCount":1,"isUnique":false,"isIndexable":false},{"name":"fileSize","typeName":"long","isOptional":true,"cardinality":"SINGLE","valuesMinCount":0,"valuesMaxCount":1,"isUnique":false,"isIndexable":false},{"name":"group","typeName":"string","isOptional":true,"cardinality":"SINGLE","valuesMinCount":0,"valuesMaxCount":1,"isUnique":false,"isIndexable":true},{"name":"posixPermissions","typeName":"fs_permissions","isOptional":true,"cardinality":"SINGLE","valuesMinCount":0,"valuesMaxCount":1,"isUnique":false,"isIndexable":true}],"superTypes":["DataSet"]}]}
\ No newline at end of file
Markdown is supported
0% or
You are about to add 0 people to the discussion. Proceed with caution.
Finish editing this message first!
Please register or to comment