Commit 3ded3ee2 by Ashutosh Mestry Committed by Madhan Neethiraj

ATLAS-2460: implementation of migration-import to migrate Atlas data in Titan to JanusGraph

parent 7f5a665e
......@@ -34,4 +34,7 @@ public final class AtlasConstants {
public static final String DEFAULT_ATLAS_REST_ADDRESS = "http://localhost:21000";
public static final int ATLAS_SHUTDOWN_HOOK_PRIORITY = 30;
public static final String DEFAULT_TYPE_VERSION = "1.0";
public static final String ATLAS_MIGRATION_MODE_FILENAME = "atlas.migration.mode.filename";
public static final String ATLAS_SERVICES_ENABLED = "atlas.services.enabled";
}
......@@ -124,6 +124,9 @@ public final class Constants {
public static final String CLASSIFICATION_EDGE_STATE_PROPERTY_KEY = STATE_PROPERTY_KEY;
public static final String CLASSIFICATION_LABEL = "classifiedAs";
public static final String VERTEX_ID_IN_IMPORT_KEY = "__vIdInImport";
public static final String EDGE_ID_IN_IMPORT_KEY = "__eIdInImport";
private Constants() {
}
......
......@@ -6,9 +6,9 @@
* to you under the Apache License, Version 2.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.
......@@ -19,6 +19,7 @@ package org.apache.atlas.service;
import org.apache.atlas.annotation.AtlasService;
import org.apache.commons.configuration.Configuration;
import org.apache.commons.lang.StringUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.springframework.context.annotation.Profile;
......@@ -28,6 +29,9 @@ import javax.annotation.PreDestroy;
import javax.inject.Inject;
import java.util.List;
import static org.apache.atlas.AtlasConstants.ATLAS_MIGRATION_MODE_FILENAME;
import static org.apache.atlas.AtlasConstants.ATLAS_SERVICES_ENABLED;
/**
* Utility for starting and stopping all services.
*/
......@@ -35,45 +39,65 @@ import java.util.List;
@Profile("!test")
public class Services {
public static final Logger LOG = LoggerFactory.getLogger(Services.class);
private static final String DATA_MIGRATION_CLASS_NAME_DEFAULT = "DataMigrationService";
private final List<Service> services;
private final Configuration configuration;
private final String dataMigrationClassName;
private final boolean servicesEnabled;
private final boolean migrationEnabled;
@Inject
public Services(List<Service> services, Configuration configuration) {
this.services = services;
this.configuration = configuration;
this.dataMigrationClassName = configuration.getString("atlas.migration.class.name", DATA_MIGRATION_CLASS_NAME_DEFAULT);
this.servicesEnabled = configuration.getBoolean(ATLAS_SERVICES_ENABLED, true);
this.migrationEnabled = StringUtils.isNotEmpty(configuration.getString(ATLAS_MIGRATION_MODE_FILENAME));
}
@PostConstruct
public void start() {
if (configuration.getBoolean("atlas.services.enabled", true)) {
try {
for (Service service : services) {
LOG.info("Starting service {}", service.getClass().getName());
for (Service svc : services) {
if (!isServiceUsed(svc)) {
continue;
}
LOG.info("Starting service {}", svc.getClass().getName());
service.start();
svc.start();
}
} catch (Exception e) {
throw new RuntimeException(e);
}
}
}
@PreDestroy
public void stop() {
if (configuration.getBoolean("atlas.services.enabled", true)) {
for (int idx = services.size() - 1; idx >= 0; idx--) {
Service service = services.get(idx);
Service svc = services.get(idx);
try {
if (!isServiceUsed(svc)) {
continue;
}
LOG.info("Stopping service {}", service.getClass().getName());
LOG.info("Stopping service {}", svc.getClass().getName());
try {
service.stop();
svc.stop();
} catch (Throwable e) {
LOG.warn("Error stopping service {}", service.getClass().getName(), e);
LOG.warn("Error stopping service {}", svc.getClass().getName(), e);
}
}
}
private boolean isServiceUsed(Service service) {
if (isDataMigrationService(service)) {
return migrationEnabled;
} else {
return !migrationEnabled && servicesEnabled;
}
}
private boolean isDataMigrationService(Service svc) {
return svc.getClass().getSimpleName().equals(dataMigrationClassName);
}
}
......@@ -18,6 +18,7 @@
package org.apache.atlas.repository.graphdb;
import java.io.IOException;
import java.io.InputStream;
import java.io.OutputStream;
import java.util.Map;
import java.util.Set;
......@@ -317,4 +318,6 @@ public interface AtlasGraph<V, E> {
* @return
*/
boolean isMultiProperty(String name);
void loadLegacyGraphSON(Map<String, String> relationshipCache, InputStream fs) throws AtlasBaseException;
}
......@@ -64,6 +64,7 @@ import javax.script.Bindings;
import javax.script.ScriptEngine;
import javax.script.ScriptException;
import java.io.IOException;
import java.io.InputStream;
import java.io.OutputStream;
import java.util.Collection;
import java.util.HashSet;
......@@ -464,4 +465,9 @@ public class AtlasJanusGraph implements AtlasGraph<AtlasJanusVertex, AtlasJanusE
}
}
}
@Override
public void loadLegacyGraphSON(Map<String, String> relationshipCache, InputStream fs) throws AtlasBaseException {
AtlasJanusGraphDatabase.loadLegacyGraphSON(relationshipCache, fs);
}
}
......@@ -20,13 +20,16 @@ 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.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.serializer.BigDecimalSerializer;
import org.apache.atlas.repository.graphdb.janus.serializer.BigIntegerSerializer;
import org.apache.atlas.repository.graphdb.janus.serializer.TypeCategorySerializer;
import org.apache.atlas.runner.LocalSolrRunner;
import org.apache.atlas.typesystem.types.DataTypes.TypeCategory;
import org.apache.atlas.utils.AtlasPerfTracer;
import org.apache.commons.configuration.Configuration;
import org.apache.tinkerpop.gremlin.structure.io.graphson.GraphSONMapper;
import org.janusgraph.graphdb.database.serialize.attribute.SerializableSerializer;
......@@ -35,22 +38,22 @@ import org.slf4j.LoggerFactory;
import org.janusgraph.core.JanusGraphFactory;
import org.janusgraph.core.JanusGraph;
import org.janusgraph.core.JanusGraphFactory;
import org.janusgraph.core.schema.JanusGraphManagement;
import org.janusgraph.graphdb.tinkerpop.JanusGraphIoRegistry;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.InputStream;
import java.math.BigDecimal;
import java.math.BigInteger;
import java.util.ArrayList;
import java.util.Map;
/**
* Default implementation for Graph Provider that doles out Titan Graph.
*/
public class AtlasJanusGraphDatabase implements GraphDatabase<AtlasJanusVertex, AtlasJanusEdge> {
private static final Logger LOG = LoggerFactory.getLogger(AtlasJanusGraphDatabase.class);
private static final Logger PERF_LOG = AtlasPerfTracer.getPerfLogger("AtlasJanusGraphDatabase");
/**
* Constant for the configuration property that indicates the prefix.
......@@ -116,6 +119,20 @@ public class AtlasJanusGraphDatabase implements GraphDatabase<AtlasJanusVertex,
return graphInstance;
}
public static JanusGraph getBulkLoadingGraphInstance() {
try {
Configuration cfg = getConfiguration();
cfg.setProperty("storage.batch-loading", true);
return JanusGraphFactory.open(cfg);
} catch (IllegalArgumentException ex) {
LOG.error("getBulkLoadingGraphInstance: Failed!", ex);
} catch (AtlasException ex) {
LOG.error("getBulkLoadingGraphInstance: Failed!", ex);
}
return null;
}
public static void unload() {
synchronized (AtlasJanusGraphDatabase.class) {
......@@ -215,4 +232,31 @@ public class AtlasJanusGraphDatabase implements GraphDatabase<AtlasJanusVertex,
return ret;
}
public static void loadLegacyGraphSON(Map<String, String> relationshipCache, InputStream fs) throws AtlasBaseException {
AtlasPerfTracer perf = null;
try {
LOG.info("Starting loadLegacyGraphSON...");
if (AtlasPerfTracer.isPerfTraceEnabled(PERF_LOG)) {
perf = AtlasPerfTracer.getPerfTracer(PERF_LOG, "loadLegacyGraphSON");
}
AtlasGraphSONReader legacyGraphSONReader = AtlasGraphSONReader.build().
relationshipCache(relationshipCache).
schemaDB(getGraphInstance()).
bulkLoadingDB(getBulkLoadingGraphInstance()).create();
legacyGraphSONReader.readGraph(fs);
} catch (Exception ex) {
LOG.error("Error loading loadLegacyGraphSON2", ex);
throw new AtlasBaseException(ex);
} finally {
AtlasPerfTracer.log(perf);
LOG.info("Done! loadLegacyGraphSON.");
}
}
}
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.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.ApplicationProperties;
import org.apache.atlas.AtlasException;
import org.apache.atlas.repository.graphdb.janus.migration.JsonNodeParsers.ParseElement;
import org.apache.atlas.repository.graphdb.janus.migration.JsonNodeProcessManager.WorkItemManager;
import org.apache.tinkerpop.gremlin.process.traversal.dsl.graph.GraphTraversal;
import org.apache.tinkerpop.gremlin.structure.Graph;
import org.apache.tinkerpop.gremlin.structure.Vertex;
import org.apache.tinkerpop.gremlin.structure.io.graphson.GraphSONMapper;
import org.apache.tinkerpop.shaded.jackson.core.JsonFactory;
import org.apache.tinkerpop.shaded.jackson.core.JsonParser;
import org.apache.tinkerpop.shaded.jackson.core.JsonToken;
import org.apache.tinkerpop.shaded.jackson.databind.JsonNode;
import org.apache.tinkerpop.shaded.jackson.databind.ObjectMapper;
import org.janusgraph.core.JanusGraph;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.IOException;
import java.io.InputStream;
import java.util.Map;
import java.util.concurrent.atomic.AtomicLong;
public final class AtlasGraphSONReader {
private static final Logger LOG = LoggerFactory.getLogger(AtlasGraphSONReader.class);
private final ObjectMapper mapper;
private final RelationshipTypeCache relationshipCache;
private final Graph graph;
private final Graph bulkLoadGraph;
private final int numWorkers;
private final int batchSize;
private final long suppliedStartIndex;
private final String[] propertiesToPostProcess;
private GraphSONUtility graphSONUtility;
private ReaderStatusManager readerStatusManager;
private AtomicLong counter;
private AtlasGraphSONReader(ObjectMapper mapper, Map<String, String> relationshipLookup, Graph graph,
Graph bulkLoadGraph, String[] propertiesToPostProcess, int numWorkers, int batchSize, long suppliedStartIndex) {
this.mapper = mapper;
this.relationshipCache = new RelationshipTypeCache(relationshipLookup);
this.graph = graph;
this.bulkLoadGraph = bulkLoadGraph;
this.numWorkers = numWorkers;
this.batchSize = batchSize;
this.suppliedStartIndex = suppliedStartIndex;
this.propertiesToPostProcess = propertiesToPostProcess;
}
public void readGraph(final InputStream inputStream) throws IOException {
counter = new AtomicLong(0);
graphSONUtility = new GraphSONUtility(relationshipCache);
final long startIndex = initStatusManager();
final JsonFactory factory = mapper.getFactory();
LOG.info("AtlasGraphSONReader.readGraph: numWorkers: {}: batchSize: {}: startIndex: {}", numWorkers, batchSize, startIndex);
try (JsonParser parser = factory.createParser(inputStream)) {
if (parser.nextToken() != JsonToken.START_OBJECT) {
throw new IOException("Expected data to start with an Object");
}
readerStatusManager.update(bulkLoadGraph, counter.get(), ReaderStatusManager.STATUS_IN_PROGRESS);
while (parser.nextToken() != JsonToken.END_OBJECT) {
final String fieldName = parser.getCurrentName() == null ? "" : parser.getCurrentName();
switch (fieldName) {
case GraphSONTokensTP2.MODE:
parser.nextToken();
final String mode = parser.getText();
if (!mode.equals("EXTENDED")) {
throw new IllegalStateException("The legacy GraphSON must be generated with GraphSONMode.EXTENDED");
}
break;
case GraphSONTokensTP2.VERTICES:
processElement(parser, new JsonNodeParsers.ParseVertex(), startIndex);
break;
case GraphSONTokensTP2.EDGES:
processElement(parser, new JsonNodeParsers.ParseEdge(), startIndex);
break;
default:
throw new IllegalStateException(String.format("Unexpected token in GraphSON - %s", fieldName));
}
}
postProcess(startIndex);
readerStatusManager.end(bulkLoadGraph, counter.get(), ReaderStatusManager.STATUS_SUCCESS);
} catch (Exception ex) {
readerStatusManager.end(bulkLoadGraph, counter.get(), ReaderStatusManager.STATUS_FAILED);
throw new IOException(ex);
} finally {
LOG.info("AtlasGraphSONReader.readGraph: Done!: {}", counter.get());
}
}
private long initStatusManager() {
readerStatusManager = new ReaderStatusManager(graph, bulkLoadGraph);
return (this.suppliedStartIndex == 0) ? readerStatusManager.getStartIndex() : this.suppliedStartIndex;
}
private void processElement(JsonParser parser, ParseElement parseElement, long startIndex) throws InterruptedException {
try {
parseElement.setContext(graphSONUtility);
WorkItemManager wim = JsonNodeProcessManager.create(graph, bulkLoadGraph, parseElement, numWorkers, batchSize, startIndex);
parser.nextToken();
while (parser.nextToken() != JsonToken.END_ARRAY) {
handleInterrupt(bulkLoadGraph, counter.incrementAndGet());
final JsonNode node = parser.readValueAsTree();
if (shouldSkip(startIndex, counter.get()) || parseElement.isTypeNode(node)) {
continue;
}
updateStatusConditionally(bulkLoadGraph, counter.get());
wim.produce(node);
}
wim.shutdown();
} catch (InterruptedException ex) {
throw ex;
} catch (Exception ex) {
LOG.error("processElement: {}: failed!", parseElement.getMessage(), ex);
} finally {
LOG.info("processElement: {}: Done! : [{}]", parseElement.getMessage(), counter);
readerStatusManager.update(bulkLoadGraph, counter.get());
}
}
private void postProcess(long startIndex) {
LOG.info("postProcess: Starting...");
try {
PostProcessManager.WorkItemsManager wim = PostProcessManager.create(bulkLoadGraph, graphSONUtility,
propertiesToPostProcess, batchSize, numWorkers);
GraphTraversal query = bulkLoadGraph.traversal().V();
while (query.hasNext()) {
handleInterrupt(bulkLoadGraph, counter.incrementAndGet());
if(shouldSkip(startIndex, counter.get())) {
continue;
}
Vertex v = (Vertex) query.next();
updateStatusConditionally(bulkLoadGraph, counter.get());
wim.produce(v.id());
}
wim.shutdown();
} catch (Exception ex) {
LOG.error("postProcess: failed!", ex);
} finally {
LOG.info("postProcess: Done! : [{}]", counter.get());
readerStatusManager.update(bulkLoadGraph, counter.get());
}
}
private boolean shouldSkip(long startIndex, long index) {
return (startIndex != 0) && (index <= startIndex);
}
private void handleInterrupt(Graph graph, long counter) throws InterruptedException {
if (!Thread.interrupted()) {
return;
}
readerStatusManager.update(graph, counter);
LOG.error("Thread interrupted: {}", counter);
throw new InterruptedException();
}
private void updateStatusConditionally(Graph graph, long counter) {
if(counter % batchSize == 0) {
readerStatusManager.update(graph, counter);
}
}
public static Builder build() throws AtlasException {
return new Builder();
}
public final static class Builder {
private int batchSize = 500;
private Map<String, String> relationshipCache;
private Graph graph;
private Graph bulkLoadGraph;
private int numWorkers;
private long suppliedStartIndex;
private String[] propertiesToPostProcess;
private Builder() {
}
private void setDefaults() throws AtlasException {
this.startIndex(ApplicationProperties.get().getLong("atlas.migration.mode.start.index", 0L))
.numWorkers(ApplicationProperties.get().getInt("atlas.migration.mode.workers", 4))
.batchSize(ApplicationProperties.get().getInt("atlas.migration.mode.batch.size", 3000))
.propertiesToPostProcess(getPropertiesToPostProcess("atlas.migration.mode.postprocess.properties"));
}
public AtlasGraphSONReader create() throws AtlasException {
setDefaults();
if(bulkLoadGraph == null) {
bulkLoadGraph = graph;
}
final GraphSONMapper.Builder builder = GraphSONMapper.build();
final GraphSONMapper mapper = builder.embedTypes(false).create();
return new AtlasGraphSONReader(mapper.createMapper(), relationshipCache, graph, bulkLoadGraph,
propertiesToPostProcess, numWorkers, batchSize, suppliedStartIndex);
}
public Builder relationshipCache(Map<String, String> relationshipCache) {
this.relationshipCache = relationshipCache;
return this;
}
public Builder schemaDB(JanusGraph graph) {
this.graph = graph;
return this;
}
public Builder bulkLoadingDB(Graph graph) {
this.bulkLoadGraph = graph;
return this;
}
public Builder numWorkers(int numWorkers) {
if(bulkLoadGraph == null || graph == null) {
this.numWorkers = 1;
LOG.info("numWorkers: {}, since one of the 2 graphs is null.", this.numWorkers);
} else {
this.numWorkers = numWorkers;
}
return this;
}
public Builder batchSize(int batchSize) {
this.batchSize = batchSize;
return this;
}
public Builder startIndex(long suppliedStartIndex) {
this.suppliedStartIndex = suppliedStartIndex;
return this;
}
public Builder propertiesToPostProcess(String[] list) {
this.propertiesToPostProcess = list;
return this;
}
private static String[] getPropertiesToPostProcess(String applicationPropertyKey) throws AtlasException {
final String HIVE_COLUMNS_PROPERTY = "hive_table.columns";
final String HIVE_PARTITION_KEYS_PROPERTY = "hive_table.partitionKeys";
final String PROCESS_INPUT_PROPERTY = "Process.inputs";
final String PROCESS_OUTPUT_PROPERTY = "Process.outputs";
final String USER_PROFILE_OUTPUT_PROPERTY = "__AtlasUserProfile.savedSearches";
String[] defaultProperties = new String[] { HIVE_COLUMNS_PROPERTY, HIVE_PARTITION_KEYS_PROPERTY,
PROCESS_INPUT_PROPERTY, PROCESS_OUTPUT_PROPERTY,
USER_PROFILE_OUTPUT_PROPERTY
};
String[] userDefinedList = ApplicationProperties.get().getStringArray(applicationPropertyKey);
return (userDefinedList == null || userDefinedList.length == 0) ? defaultProperties : userDefinedList;
}
}
}
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS 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;
public final class GraphSONTokensTP2 {
public static final String _ID = "_id";
public static final String _LABEL = "_label";
public static final String _TYPE = "_type";
public static final String _OUT_V = "_outV";
public static final String _IN_V = "_inV";
public static final String VALUE = "value";
public static final String TYPE = "type";
public static final String TYPE_LIST = "list";
public static final String TYPE_STRING = "string";
public static final String TYPE_DOUBLE = "double";
public static final String TYPE_INTEGER = "integer";
public static final String TYPE_FLOAT = "float";
public static final String TYPE_MAP = "map";
public static final String TYPE_BOOLEAN = "boolean";
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_UNKNOWN = "unknown";
public static final String VERTICES = "vertices";
public static final String EDGES = "edges";
public static final String MODE = "mode";
private GraphSONTokensTP2() {
}
}
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.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 com.google.common.annotations.VisibleForTesting;
import org.apache.atlas.repository.Constants;
import org.apache.commons.lang.StringUtils;
import org.apache.tinkerpop.gremlin.structure.Edge;
import org.apache.tinkerpop.gremlin.structure.Graph;
import org.apache.tinkerpop.gremlin.structure.Graph.Features.EdgeFeatures;
import org.apache.tinkerpop.gremlin.structure.Graph.Features.VertexFeatures;
import org.apache.tinkerpop.gremlin.structure.T;
import org.apache.tinkerpop.gremlin.structure.Vertex;
import org.apache.tinkerpop.shaded.jackson.databind.JsonNode;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.util.*;
import static org.apache.atlas.repository.Constants.CLASSIFICATION_EDGE_STATE_PROPERTY_KEY;
import static org.apache.atlas.repository.Constants.CLASSIFICATION_ENTITY_GUID;
import static org.apache.atlas.repository.Constants.RELATIONSHIPTYPE_TAG_PROPAGATION_KEY;
class GraphSONUtility {
private static final Logger LOG = LoggerFactory.getLogger(GraphSONUtility.class);
private static final String EMPTY_STRING = "";
private final RelationshipTypeCache relationshipTypeCache;
public GraphSONUtility(final RelationshipTypeCache relationshipTypeCache) {
this.relationshipTypeCache = relationshipTypeCache;
}
public Map<String, Object> vertexFromJson(Graph g, final JsonNode json) {
final Map<String, Object> props = readProperties(json);
if (props.containsKey(Constants.TYPENAME_PROPERTY_KEY)) {
return null;
}
Map<String, Object> schemaUpdate = null;
VertexFeatures vertexFeatures = g.features().vertex();
Object vertexId = getTypedValueFromJsonNode(json.get(GraphSONTokensTP2._ID));
Vertex vertex = vertexFeatures.willAllowId(vertexId) ? g.addVertex(T.id, vertexId) : g.addVertex();
props.put(Constants.VERTEX_ID_IN_IMPORT_KEY, vertexId);
for (Map.Entry<String, Object> entry : props.entrySet()) {
try {
vertex.property(vertexFeatures.getCardinality(entry.getKey()), entry.getKey(), entry.getValue());
} catch (IllegalArgumentException ex) {
schemaUpdate = getSchemaUpdateMap(schemaUpdate);
if (!schemaUpdate.containsKey("id")) {
schemaUpdate.put("id", vertex.id());
}
schemaUpdate.put(entry.getKey(), entry.getValue());
}
}
return schemaUpdate;
}
public Map<String, Object> edgeFromJson(Graph g, MappedElementCache cache, final JsonNode json) {
final JsonNode nodeLabel = json.get(GraphSONTokensTP2._LABEL);
String label = nodeLabel == null ? EMPTY_STRING : nodeLabel.textValue();
if (label.startsWith("__type.")) {
return null;
}
Map<String, Object> schemaUpdate = null;
Object edgeId = null;
try {
final Vertex in = getMappedVertex(g, cache, json, GraphSONTokensTP2._IN_V);
final Vertex out = getMappedVertex(g, cache, json, GraphSONTokensTP2._OUT_V);
if (in == null || out == null) {
return null;
}
edgeId = getTypedValueFromJsonNode(json.get(GraphSONTokensTP2._ID));
final Map<String, Object> props = GraphSONUtility.readProperties(json);
props.put(Constants.EDGE_ID_IN_IMPORT_KEY, edgeId.toString());
if(addRelationshipTypeForClassification(in, out, label, props)) {
label = Constants.CLASSIFICATION_LABEL;
} else {
addRelationshipTypeName(label, props);
}
EdgeFeatures edgeFeatures = g.features().edge();
final Edge edge = edgeFeatures.willAllowId(edgeId) ? out.addEdge(label, in, T.id, edgeId) : out.addEdge(label, in);
addMandatoryRelationshipProperties(props);
for (Map.Entry<String, Object> entry : props.entrySet()) {
try {
edge.property(entry.getKey(), entry.getValue());
} catch (IllegalArgumentException ex) {
schemaUpdate = getSchemaUpdateMap(schemaUpdate);
if (!schemaUpdate.containsKey("id")) {
schemaUpdate.put("id", edge.id());
}
schemaUpdate.put(entry.getKey(), entry.getValue());
}
}
} catch(IllegalArgumentException ex) {
schemaUpdate = getSchemaUpdateMap(schemaUpdate);
schemaUpdate.put("oid", edgeId);
}
return schemaUpdate;
}
private Map<String, Object> getSchemaUpdateMap(Map<String, Object> schemaUpdate) {
if(schemaUpdate == null) {
schemaUpdate = new HashMap<>();
}
return schemaUpdate;
}
private Vertex getMappedVertex(Graph gr, MappedElementCache cache, JsonNode json, String direction) {
Object inVId = GraphSONUtility.getTypedValueFromJsonNode(json.get(direction));
return cache.getMappedVertex(gr, inVId);
}
private boolean addRelationshipTypeForClassification(Vertex in, Vertex out, String label, Map<String, Object> props) {
if (in.property(Constants.ENTITY_TYPE_PROPERTY_KEY).isPresent()) {
String inTypeName = (String) in.property(Constants.ENTITY_TYPE_PROPERTY_KEY).value();
if (inTypeName.equals(label)) {
if (StringUtils.isNotEmpty(inTypeName)) {
props.put(Constants.ENTITY_TYPE_PROPERTY_KEY, inTypeName);
addEntityGuidToTrait(in, out);
return true;
} else {
LOG.info("Could not find typeName for trait: {}", label);
}
}
}
return false;
}
private void addEntityGuidToTrait(Vertex in, Vertex out) {
String entityGuid = "";
if (out.property(Constants.GUID_PROPERTY_KEY).isPresent()) {
entityGuid = (String) out.property(Constants.GUID_PROPERTY_KEY).value();
}
if(StringUtils.isNotEmpty(entityGuid)) {
in.property(CLASSIFICATION_ENTITY_GUID, entityGuid);
}
}
private void addRelationshipTypeName(String edgeLabel, Map<String, Object> props) {
String typeName = relationshipTypeCache.get(edgeLabel);
if (StringUtils.isNotEmpty(typeName)) {
props.put(Constants.ENTITY_TYPE_PROPERTY_KEY, typeName);
} else {
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());
props.put(RELATIONSHIPTYPE_TAG_PROPAGATION_KEY, "NONE");
props.put(CLASSIFICATION_EDGE_STATE_PROPERTY_KEY, "ACTIVE");
}
public void replaceReferencedEdgeIdForList(Graph g, MappedElementCache cache, Vertex v, String propertyName) {
try {
if (v.property(Constants.TYPENAME_PROPERTY_KEY).isPresent() || !v.property(propertyName).isPresent()) {
return;
}
List list = (List) v.property(propertyName).value();
for (int i = 0; i < list.size(); i++) {
String id = list.get(i).toString();
Object newId = cache.getMappedEdge(g, id);
if (newId == null) {
continue;
}
list.set(i, newId.toString());
}
v.property(propertyName, list);
} catch (IllegalArgumentException ex) {
if (LOG.isDebugEnabled()) {
LOG.debug("processItem: IllegalArgumentException: v[{}] error!", v.id(), ex);
}
}
}
static Map<String, Object> readProperties(final JsonNode node) {
final Map<String, Object> map = new HashMap<>();
final Iterator<Map.Entry<String, JsonNode>> iterator = node.fields();
while (iterator.hasNext()) {
final Map.Entry<String, JsonNode> entry = iterator.next();
if (!isReservedKey(entry.getKey())) {
// it generally shouldn't be as such but graphson containing null values can't be shoved into
// element property keys or it will result in error
final Object o = readProperty(entry.getValue());
if (o != null) {
map.put(entry.getKey(), o);
}
}
}
return map;
}
private static boolean isReservedKey(final String key) {
return key.equals(GraphSONTokensTP2._ID) || key.equals(GraphSONTokensTP2._TYPE) || key.equals(GraphSONTokensTP2._LABEL)
|| key.equals(GraphSONTokensTP2._OUT_V) || key.equals(GraphSONTokensTP2._IN_V);
}
private static Object readProperty(final JsonNode node) {
final Object propertyValue;
if (node.get(GraphSONTokensTP2.TYPE).textValue().equals(GraphSONTokensTP2.TYPE_UNKNOWN)) {
propertyValue = null;
} else if (node.get(GraphSONTokensTP2.TYPE).textValue().equals(GraphSONTokensTP2.TYPE_BOOLEAN)) {
propertyValue = node.get(GraphSONTokensTP2.VALUE).booleanValue();
} else if (node.get(GraphSONTokensTP2.TYPE).textValue().equals(GraphSONTokensTP2.TYPE_FLOAT)) {
propertyValue = Float.parseFloat(node.get(GraphSONTokensTP2.VALUE).asText());
} else if (node.get(GraphSONTokensTP2.TYPE).textValue().equals(GraphSONTokensTP2.TYPE_BYTE)) {
propertyValue = Byte.parseByte(node.get(GraphSONTokensTP2.VALUE).asText());
} else if (node.get(GraphSONTokensTP2.TYPE).textValue().equals(GraphSONTokensTP2.TYPE_SHORT)) {
propertyValue = Short.parseShort(node.get(GraphSONTokensTP2.VALUE).asText());
} else if (node.get(GraphSONTokensTP2.TYPE).textValue().equals(GraphSONTokensTP2.TYPE_DOUBLE)) {
propertyValue = node.get(GraphSONTokensTP2.VALUE).doubleValue();
} 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();
} 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)) {
propertyValue = readProperties(node.get(GraphSONTokensTP2.VALUE).elements());
} else if (node.get(GraphSONTokensTP2.TYPE).textValue().equals(GraphSONTokensTP2.TYPE_MAP)) {
propertyValue = readProperties(node.get(GraphSONTokensTP2.VALUE));
} else {
propertyValue = node.textValue();
}
return propertyValue;
}
private static List readProperties(final Iterator<JsonNode> listOfNodes) {
final List<Object> array = new ArrayList<>();
while (listOfNodes.hasNext()) {
array.add(readProperty(listOfNodes.next()));
}
return array;
}
@VisibleForTesting
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;
}
}
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS 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.Constants;
import org.apache.tinkerpop.gremlin.structure.Element;
import org.apache.tinkerpop.gremlin.structure.Graph;
import org.apache.tinkerpop.shaded.jackson.databind.JsonNode;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
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;
public class JsonNodeParsers {
private static final Logger LOG = LoggerFactory.getLogger(JsonNodeParsers.class);
static abstract class ParseElement {
protected GraphSONUtility utility;
abstract String getMessage();
abstract Object getId(JsonNode node);
abstract boolean isTypeNode(JsonNode node);
abstract String getType(JsonNode node);
public void setContext(GraphSONUtility utility) {
this.utility = utility;
}
abstract Map<String, Object> parse(Graph gr, MappedElementCache cache, JsonNode Node);
public void commit(Graph graph) {
graph.tx().commit();
}
abstract Element get(Graph gr, Object id);
abstract Element getByOriginalId(Graph gr, Object id);
public Element getByOriginalId(Graph gr, JsonNode node) {
return getByOriginalId(gr, getId(node));
}
public Element update(Graph gr, Object id, Map<String,Object> schema) {
Element el = get(gr, id);
for (Map.Entry<String, Object> entry : schema.entrySet()) {
el.property(entry.getKey(), entry.getValue());
}
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
public String getMessage() {
return MESSAGE_EDGE;
}
@Override
Object getId(JsonNode node) {
return getTypedValueFromJsonNode(node.get(GraphSONTokensTP2._ID));
}
@Override
public Map<String, Object> parse(Graph gr, MappedElementCache cache, JsonNode node) {
return utility.edgeFromJson(gr, cache, node);
}
@Override
Element get(Graph gr, Object id) {
return gr.edges(id).next();
}
@Override
Element getByOriginalId(Graph gr, Object id) {
try {
return gr.traversal().E().has(EDGE_ID_IN_IMPORT_KEY, id).next();
} catch (Exception ex) {
LOG.error("fetchEdge: fetchFromDB failed: {}", id);
return null;
}
}
@Override
public boolean isTypeNode(JsonNode node) {
return node.get(GraphSONTokensTP2._LABEL).textValue().startsWith(Constants.TYPENAME_PROPERTY_KEY);
}
@Override
public String getType(JsonNode node) {
return node.get(GraphSONTokensTP2._LABEL).textValue();
}
}
static class ParseVertex extends ParseElement {
private static final String NODE_VALUE_KEY = "value";
private static final String MESSAGE_VERTEX = "vertex";
@Override
public String getMessage() {
return MESSAGE_VERTEX;
}
@Override
Object getId(JsonNode node) {
return getTypedValueFromJsonNode(node.get(GraphSONTokensTP2._ID));
}
@Override
public Map<String, Object> parse(Graph graph, MappedElementCache cache, JsonNode node) {
return utility.vertexFromJson(graph, node);
}
@Override
Element get(Graph gr, Object id) {
return gr.vertices(id).next();
}
@Override
Element getByOriginalId(Graph gr, Object id) {
try {
return gr.traversal().V().has(VERTEX_ID_IN_IMPORT_KEY, id).next();
} catch (Exception ex) {
LOG.error("getByOriginalId failed: {}", id);
return null;
}
}
@Override
public boolean isTypeNode(JsonNode node) {
return node.has(Constants.TYPENAME_PROPERTY_KEY);
}
@Override
public String getType(JsonNode node) {
return node.has(Constants.ENTITY_TYPE_PROPERTY_KEY) ? node.get(Constants.ENTITY_TYPE_PROPERTY_KEY).get(NODE_VALUE_KEY).toString() : "";
}
}
}
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.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.shaded.jackson.databind.JsonNode;
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.JsonNodeParsers.ParseElement;
import org.apache.tinkerpop.gremlin.structure.Graph;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.util.ArrayList;
import java.util.List;
import java.util.Map;
import java.util.NoSuchElementException;
import java.util.concurrent.BlockingQueue;
public class JsonNodeProcessManager {
private static class Consumer extends WorkItemConsumer<JsonNode> {
private static final Logger LOG = LoggerFactory.getLogger(Consumer.class);
private static final int WAIT_DURATION_AFTER_COMMIT_EXCEPTION = 1000;
private final Graph graph;
protected final Graph bulkLoadGraph;
protected final ParseElement parseElement;
private final long batchSize;
private long counter;
private final MappedElementCache cache;
private final List<JsonNode> nodes = new ArrayList<>();
public Consumer(BlockingQueue<JsonNode> workQueue, Graph graph, Graph bulkLoadGraph, ParseElement parseElement, long batchSize) {
super(workQueue);
this.graph = graph;
this.bulkLoadGraph = bulkLoadGraph;
this.parseElement = parseElement;
this.batchSize = batchSize;
this.counter = 0;
this.cache = new MappedElementCache();
}
@Override
public void processItem(JsonNode node) {
try {
Map<String, Object> result = parseElement.parse(bulkLoadGraph, cache, node);
if (result == null) {
nodes.add(node);
commitConditionally(counter++);
} else {
commitBulk();
cache.clearAll();
updateSchema(result, node);
}
} catch (Exception ex) {
bulkLoadGraph.tx().rollback();
error("Failed! Retrying...", ex);
retryBatchCommit();
}
}
@Override
protected void commitDirty() {
super.commitDirty();
cache.clearAll();
}
@Override
protected void doCommit() {
commitBulk();
}
private void commitConditionally(long index) {
if (index % batchSize == 0 && nodes.size() > 0) {
commitBulk();
}
}
private void commitBulk() {
commit(bulkLoadGraph, nodes.size());
nodes.clear();
}
private void commitRegular() {
commit(graph, nodes.size());
}
private void commit(Graph g, int size) {
parseElement.commit(g);
display("commit-size: {}: Done!", size);
}
private void updateSchema(Map<String, Object> schema, org.apache.tinkerpop.shaded.jackson.databind.JsonNode node) {
synchronized (graph) {
String typeName = parseElement.getType(node);
try {
display("updateSchema: type: {}: ...", typeName);
if (schema.containsKey("oid")) {
parseElement.parse(graph, cache, node);
} else {
Object id = schema.get("id");
schema.remove("id");
parseElement.update(graph, id, schema);
}
commitRegular();
display("updateSchema: type: {}: Done!", typeName);
} catch (NoSuchElementException ex) {
parseElement.parse(graph, cache, node);
commitRegular();
display("updateSchema: NoSuchElementException processed!: type: {}: Done!", typeName);
} catch (Exception ex) {
graph.tx().rollback();
error("updateSchema: failed!: type: " + typeName, ex);
}
}
}
private void retryBatchCommit() {
display("Waiting with [{} nodes] for 1 secs.", nodes.size());
try {
Thread.sleep(WAIT_DURATION_AFTER_COMMIT_EXCEPTION);
for (org.apache.tinkerpop.shaded.jackson.databind.JsonNode n : nodes) {
parseElement.parse(bulkLoadGraph, cache, n);
}
commitBulk();
display("Done!: After re-adding {}.", nodes.size());
} catch (Exception ex) {
error("retryBatchCommit: Failed! Potential data loss.", ex);
}
}
private void display(String message, Object s1, Object s2) {
LOG.info("{}: [{}]: " + message, parseElement.getMessage(), counter, s1, s2);
}
private void display(String message, Object s1) {
display(message, s1, "");
}
private void error(String message, Exception ex) {
LOG.error("{}: [{}]: " + message, parseElement.getMessage(), counter, ex);
}
}
private static class ResumingConsumer extends Consumer {
public ResumingConsumer(BlockingQueue<JsonNode> workQueue, Graph graph, Graph bulkLoadGraph, ParseElement parseElement, long batchSize) {
super(workQueue, graph, bulkLoadGraph, parseElement, batchSize);
}
@Override
public void processItem(JsonNode node) {
if (!contains(node)) {
super.processItem(node);
}
}
private boolean contains(JsonNode node) {
return (parseElement.getByOriginalId(bulkLoadGraph, node) != null);
}
}
private static class ConsumerBuilder implements WorkItemBuilder<Consumer, JsonNode> {
private final Graph graph;
private final Graph bulkLoadGraph;
private final ParseElement parseElement;
private final int batchSize;
private final long startIndex;
public ConsumerBuilder(Graph graph, Graph bulkLoadGraph, ParseElement parseElement, int batchSize, long startIndex) {
this.graph = graph;
this.bulkLoadGraph = bulkLoadGraph;
this.batchSize = batchSize;
this.parseElement = parseElement;
this.startIndex = startIndex;
}
@Override
public Consumer build(BlockingQueue<JsonNode> queue) {
if(startIndex == 0) {
return new Consumer(queue, graph, bulkLoadGraph, parseElement, batchSize);
}
return new ResumingConsumer(queue, graph, bulkLoadGraph, parseElement, batchSize);
}
}
static class WorkItemManager extends org.apache.atlas.repository.graphdb.janus.migration.pc.WorkItemManager {
public WorkItemManager(WorkItemBuilder builder, int batchSize, int numWorkers) {
super(builder, batchSize, numWorkers);
}
}
public static WorkItemManager create(Graph rGraph, Graph bGraph,
ParseElement parseElement, int numWorkers, int batchSize, long startIndex) {
ConsumerBuilder cb = new ConsumerBuilder(rGraph, bGraph, parseElement, batchSize, startIndex);
return new WorkItemManager(cb, batchSize, numWorkers);
}
}
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.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.utils.LruCache;
import org.apache.tinkerpop.gremlin.structure.Edge;
import org.apache.tinkerpop.gremlin.structure.Graph;
import org.apache.tinkerpop.gremlin.structure.Vertex;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
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;
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);
public Vertex getMappedVertex(Graph gr, Object key) {
try {
Vertex ret = lruVertexCache.get(key);
if (ret == null) {
synchronized (lruVertexCache) {
ret = lruVertexCache.get(key);
if(ret == null) {
ret = fetchVertex(gr, key);
lruVertexCache.put(key, ret);
}
}
}
return ret;
} catch (Exception ex) {
LOG.error("getMappedVertex: {}", key, ex);
return null;
}
}
public String getMappedEdge(Graph gr, String key) {
try {
String ret = lruEdgeCache.get(key);
if (ret == null) {
synchronized (lruEdgeCache) {
ret = lruEdgeCache.get(key);
if (ret == null) {
Edge e = fetchEdge(gr, key);
ret = e.id().toString();
lruEdgeCache.put(key, ret);
}
}
}
return ret;
} catch (Exception ex) {
LOG.error("getMappedEdge: {}", key, ex);
return null;
}
}
private Vertex fetchVertex(Graph gr, Object key) {
try {
return gr.traversal().V().has(VERTEX_ID_IN_IMPORT_KEY, key).next();
} catch (Exception ex) {
LOG.error("fetchVertex: fetchFromDB failed: {}", key);
return null;
}
}
private Edge fetchEdge(Graph gr, String key) {
try {
return gr.traversal().E().has(EDGE_ID_IN_IMPORT_KEY, key).next();
} catch (Exception ex) {
LOG.error("fetchEdge: fetchFromDB failed: {}", key);
return null;
}
}
public void clearVertexCache() {
lruVertexCache.clear();
}
public void clearEdgeCache() {
lruEdgeCache.clear();
}
public void clearAll() {
clearVertexCache();
clearEdgeCache();
}
}
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.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.tinkerpop.gremlin.structure.Graph;
import org.apache.tinkerpop.gremlin.structure.Vertex;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.util.concurrent.BlockingQueue;
public class PostProcessManager {
private static class Consumer extends WorkItemConsumer<Object> {
private static final Logger LOG = LoggerFactory.getLogger(Consumer.class);
private final Graph bulkLoadGraph;
private final GraphSONUtility utility;
private final String[] properties;
private final MappedElementCache cache;
private final int batchSize;
private long counter;
private long batchCounter;
public Consumer(BlockingQueue<Object> queue, Graph bulkLoadGraph, GraphSONUtility utility,
String[] properties, MappedElementCache cache, int batchSize) {
super(queue);
this.bulkLoadGraph = bulkLoadGraph;
this.utility = utility;
this.properties = properties;
this.cache = cache;
this.batchSize = batchSize;
this.counter = 0;
this.batchCounter = 0;
}
@Override
public void processItem(Object vertexId) {
batchCounter++;
counter++;
try {
Vertex v = bulkLoadGraph.traversal().V(vertexId).next();
for (String p : properties) {
utility.replaceReferencedEdgeIdForList(bulkLoadGraph, cache, v, p);
}
if (batchCounter >= batchSize) {
LOG.info("[{}]: batch: {}: commit", counter, batchCounter);
commit();
batchCounter = 0;
}
}
catch (Exception ex) {
LOG.error("processItem: v[{}] error!", vertexId, ex);
}
}
@Override
protected void doCommit() {
bulkLoadGraph.tx().commit();
}
}
private static class ConsumerBuilder implements WorkItemBuilder<Consumer, Object> {
private final Graph bulkLoadGraph;
private final GraphSONUtility utility;
private final int batchSize;
private final MappedElementCache cache;
private final String[] vertexPropertiesToPostProcess;
public ConsumerBuilder(Graph bulkLoadGraph, GraphSONUtility utility, String[] propertiesToPostProcess, int batchSize) {
this.bulkLoadGraph = bulkLoadGraph;
this.utility = utility;
this.batchSize = batchSize;
this.cache = new MappedElementCache();
this.vertexPropertiesToPostProcess = propertiesToPostProcess;
}
@Override
public Consumer build(BlockingQueue<Object> queue) {
return new Consumer(queue, bulkLoadGraph, utility, vertexPropertiesToPostProcess, cache, batchSize);
}
}
static class WorkItemsManager extends WorkItemManager<Object, Consumer> {
public WorkItemsManager(WorkItemBuilder builder, int batchSize, int numWorkers) {
super(builder, batchSize, numWorkers);
}
}
public static WorkItemsManager create(Graph bGraph, GraphSONUtility utility, String[] propertiesToPostProcess, int batchSize, int numWorkers) {
ConsumerBuilder cb = new ConsumerBuilder(bGraph, utility, propertiesToPostProcess, batchSize);
return new WorkItemsManager(cb, batchSize, numWorkers);
}
}
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS 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.Constants;
import org.apache.tinkerpop.gremlin.process.traversal.dsl.graph.GraphTraversal;
import org.apache.tinkerpop.gremlin.process.traversal.dsl.graph.GraphTraversalSource;
import org.apache.tinkerpop.gremlin.structure.Graph;
import org.apache.tinkerpop.gremlin.structure.Vertex;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.util.Date;
public class ReaderStatusManager {
private static final Logger LOG = LoggerFactory.getLogger(ReaderStatusManager.class);
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";
public static final String STATUS_NOT_STARTED = "NOT STARTED";
public static final String STATUS_IN_PROGRESS = "IN PROGRESS";
public static final String STATUS_SUCCESS = "SUCCESS";
public static final String STATUS_FAILED = "FAILED";
private Object migrationStatusId = null;
private Vertex migrationStatus = null;
public ReaderStatusManager(Graph graph, Graph bulkLoadGraph) {
init(graph, bulkLoadGraph);
}
public void init(Graph graph, Graph bulkLoadGraph) {
migrationStatus = fetchUsingTypeName(bulkLoadGraph.traversal());
if(migrationStatus == null) {
createAndCommit(graph);
migrationStatus = fetchUsingId(bulkLoadGraph.traversal());
}
if(migrationStatus == null) {
migrationStatus = fetchUsingId(bulkLoadGraph.traversal());
}
}
public void end(Graph bGraph, Long counter, String status) {
migrationStatus.property(END_TIME_PROPERTY, new Date());
migrationStatus.property(TOTAL_COUNT_PROPERTY, counter);
update(bGraph, counter, status);
}
public void update(Graph graph, Long counter) {
migrationStatus.property(CURRENT_INDEX_PROPERTY, counter);
graph.tx().commit();
}
public void update(Graph graph, Long counter, String status) {
migrationStatus.property(OPERATION_STATUS_PROPERTY, status);
update(graph, counter);
}
public void clear() {
migrationStatus = null;
}
public long getStartIndex() {
return (long) migrationStatus.property(CURRENT_INDEX_PROPERTY).value();
}
private Vertex fetchUsingId(GraphTraversalSource g) {
return g.V(migrationStatusId).next();
}
private 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;
}
private void createAndCommit(Graph rGraph) {
Vertex v = rGraph.addVertex();
long longValue = 0L;
v.property(Constants.ENTITY_TYPE_PROPERTY_KEY, MIGRATION_STATUS_TYPE_NAME);
v.property(CURRENT_INDEX_PROPERTY, longValue);
v.property(TOTAL_COUNT_PROPERTY, longValue);
v.property(OPERATION_STATUS_PROPERTY, STATUS_NOT_STARTED);
v.property(START_TIME_PROPERTY, new Date());
v.property(END_TIME_PROPERTY, new Date());
migrationStatusId = v.id();
rGraph.tx().commit();
LOG.info("migrationStatus vertex created! v[{}]", migrationStatusId);
}
}
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.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.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 {
private static final Logger LOG = LoggerFactory.getLogger(RelationshipTypeCache.class);
private final Map<String, String> relationshipLookup;
public RelationshipTypeCache(Map<String, String> lookup) {
relationshipLookup = lookup;
}
public String get(String label) {
return relationshipLookup.get(label);
}
}
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS 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.pc;
import java.util.concurrent.BlockingQueue;
public interface WorkItemBuilder<T extends Runnable, U> {
T build(BlockingQueue<U> queue);
}
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.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.pc;
import com.google.common.base.Stopwatch;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.util.concurrent.BlockingQueue;
import java.util.concurrent.TimeUnit;
public abstract class WorkItemConsumer<T> implements Runnable {
private static final Logger LOG = LoggerFactory.getLogger(WorkItemConsumer.class);
private static final int POLLING_DURATION_SECONDS = 5;
private final BlockingQueue<T> queue;
private boolean isDirty = false;
private long maxCommitTimeSeconds = 0;
public WorkItemConsumer(BlockingQueue<T> queue) {
this.queue = queue;
}
public void run() {
while (!Thread.currentThread().isInterrupted()) {
try {
T item = queue.poll(POLLING_DURATION_SECONDS, TimeUnit.SECONDS);
if (item == null) {
commitDirty();
return;
}
isDirty = true;
processItem(item);
} catch (InterruptedException e) {
LOG.error("WorkItemConsumer: Interrupted: ", e);
}
}
}
public long getMaxCommitTimeSeconds() {
return (this.maxCommitTimeSeconds > 0 ? this.maxCommitTimeSeconds : 15);
}
protected void commitDirty() {
if (!isDirty) {
return;
}
LOG.info("isDirty");
commit();
}
protected void commit() {
Stopwatch sw = Stopwatch.createStarted();
doCommit();
sw.stop();
updateCommitTime(sw.elapsed(TimeUnit.SECONDS));
isDirty = false;
}
protected abstract void doCommit();
protected abstract void processItem(T item);
private void updateCommitTime(long commitTime) {
if (this.maxCommitTimeSeconds < commitTime) {
this.maxCommitTimeSeconds = commitTime;
}
}
}
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS 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.pc;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.util.ArrayList;
import java.util.List;
import java.util.concurrent.*;
public class WorkItemManager<T, U extends WorkItemConsumer> {
private static final Logger LOG = LoggerFactory.getLogger(WorkItemManager.class);
private final BlockingQueue<T> workQueue;
private final ExecutorService service;
private final List<U> consumers = new ArrayList<>();
public WorkItemManager(WorkItemBuilder builder, int batchSize, int numWorkers) {
workQueue = new LinkedBlockingQueue<>(batchSize * numWorkers);
service = Executors.newFixedThreadPool(numWorkers);
for (int i = 0; i < numWorkers; i++) {
U c = (U) builder.build(workQueue);
service.submit(c);
consumers.add(c);
}
}
public void produce(T item) {
try {
workQueue.put(item);
} catch (InterruptedException ex) {
Thread.currentThread().interrupt();
}
}
public void shutdown() throws InterruptedException {
int avgCommitTimeSeconds = getAvgCommitTimeSeconds() * 2;
LOG.info("WorkItemManager: Shutdown started. Will wait for: {} seconds...", avgCommitTimeSeconds);
service.shutdown();
service.awaitTermination(avgCommitTimeSeconds, TimeUnit.MINUTES);
LOG.info("WorkItemManager: Shutdown done!");
}
private int getAvgCommitTimeSeconds() {
int commitTimeSeconds = 0;
for (U c : consumers) {
commitTimeSeconds += c.getMaxCommitTimeSeconds();
}
return commitTimeSeconds / consumers.size();
}
}
......@@ -55,6 +55,7 @@ import javax.script.ScriptEngine;
import javax.script.ScriptEngineManager;
import javax.script.ScriptException;
import java.io.IOException;
import java.io.InputStream;
import java.io.OutputStream;
import java.util.ArrayList;
import java.util.Collection;
......@@ -414,7 +415,12 @@ public class Titan0Graph implements AtlasGraph<Titan0Vertex, Titan0Edge> {
return multiProperties.contains(propertyName);
}
@Override
public void loadLegacyGraphSON(Map<String, String> relationshipCache, InputStream fs) throws AtlasBaseException {
}
public void addMultiProperties(Set<String> names) {
multiProperties.addAll(names);
}
}
......@@ -50,7 +50,7 @@ public class AtlasExportResult implements Serializable {
public final static String ENTITY_COUNT = "entityCount";
public enum OperationStatus {
SUCCESS, PARTIAL_SUCCESS, FAIL
SUCCESS, PARTIAL_SUCCESS, INPROGRESS, FAIL
}
private AtlasExportRequest request;
......@@ -143,6 +143,9 @@ public class AtlasExportResult implements Serializable {
this.operationStatus = operationStatus;
}
public void setMetric(String key, int value) {
metrics.put(key, value);
}
public void incrementMeticsCounter(String key) {
incrementMeticsCounter(key, 1);
......@@ -222,7 +225,6 @@ public class AtlasExportResult implements Serializable {
public void setEntityCreationOrder(List<String> entityCreationOrder) { this.entityCreationOrder = entityCreationOrder; }
public StringBuilder toString(StringBuilder sb) {
if (sb == null) {
sb = new StringBuilder();
......
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS 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.model.impexp;
import com.fasterxml.jackson.annotation.JsonAutoDetect;
import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
import com.fasterxml.jackson.databind.annotation.JsonSerialize;
import java.io.Serializable;
import java.util.Date;
import static com.fasterxml.jackson.annotation.JsonAutoDetect.Visibility.NONE;
import static com.fasterxml.jackson.annotation.JsonAutoDetect.Visibility.PUBLIC_ONLY;
@JsonAutoDetect(getterVisibility = PUBLIC_ONLY, setterVisibility = PUBLIC_ONLY, fieldVisibility = NONE)
@JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
@JsonIgnoreProperties(ignoreUnknown = true)
public class MigrationStatus implements Serializable {
private static final long serialVersionUID = 1L;
private String operationStatus;
private Date startTime;
private Date endTime;
private long currentIndex;
private long totalCount;
public void setOperationStatus(String operationStatus) {
this.operationStatus = operationStatus;
}
public String getOperationStatus() {
return operationStatus;
}
public void setStartTime(Date startTime) {
this.startTime = startTime;
}
public Date getStartTime() {
return startTime;
}
public void setEndTime(Date endTime) {
this.endTime = endTime;
}
public Date getEndTime() {
return endTime;
}
public void setCurrentIndex(long currentIndex) {
this.currentIndex = currentIndex;
}
public long getCurrentIndex() {
return currentIndex;
}
public void setTotalCount(long totalCount) {
this.totalCount = totalCount;
}
public long getTotalCount() {
return this.totalCount;
}
public StringBuilder toString(StringBuilder sb) {
sb.append(", operationStatus=").append(operationStatus);
sb.append(", startTime=").append(startTime);
sb.append(", endTime=").append(endTime);
sb.append(", currentIndex=").append(currentIndex);
sb.append(", totalCount=").append(totalCount);
return sb;
}
}
......@@ -28,7 +28,9 @@ import org.apache.atlas.model.typedef.AtlasRelationshipDef;
import org.apache.atlas.model.typedef.AtlasStructDef;
import org.apache.atlas.model.typedef.AtlasTypesDef;
import java.io.InputStream;
import java.util.List;
import java.util.Map;
/**
* Interface to persistence store of TypeDef
......@@ -105,4 +107,6 @@ public interface AtlasTypeDefStore {
AtlasBaseTypeDef getByName(String name) throws AtlasBaseException;
AtlasBaseTypeDef getByGuid(String guid) throws AtlasBaseException;
void loadLegacyData(Map<String, String> relationshipCache, InputStream fs) throws AtlasBaseException;
}
......@@ -1757,7 +1757,10 @@
<version>2.7</version>
<configuration>
<encoding>UTF-8</encoding>
<nonFilteredFileExtensions>zip</nonFilteredFileExtensions>
<nonFilteredFileExtensions>
<nonFilteredFileExtension>zip</nonFilteredFileExtension>
<nonFilteredFileExtension>json</nonFilteredFileExtension>
</nonFilteredFileExtensions>
</configuration>
</plugin>
......
......@@ -18,7 +18,8 @@
~ limitations under the License.
-->
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
<modelVersion>4.0.0</modelVersion>
<parent>
......@@ -256,7 +257,7 @@
<artifactId>maven-resources-plugin</artifactId>
<executions>
<execution>
<id>copy-resources</id>
<id>copy-resources-solr</id>
<phase>validate</phase>
<goals>
<goal>copy-resources</goal>
......
......@@ -67,27 +67,7 @@ import java.util.List;
import java.util.Set;
import static org.apache.atlas.model.typedef.AtlasBaseTypeDef.*;
import static org.apache.atlas.repository.Constants.BACKING_INDEX;
import static org.apache.atlas.repository.Constants.CLASSIFICATION_EDGE_IS_PROPAGATED_PROPERTY_KEY;
import static org.apache.atlas.repository.Constants.CLASSIFICATION_LABEL;
import static org.apache.atlas.repository.Constants.CLASSIFICATION_EDGE_NAME_PROPERTY_KEY;
import static org.apache.atlas.repository.Constants.CREATED_BY_KEY;
import static org.apache.atlas.repository.Constants.EDGE_INDEX;
import static org.apache.atlas.repository.Constants.ENTITY_TEXT_PROPERTY_KEY;
import static org.apache.atlas.repository.Constants.ENTITY_TYPE_PROPERTY_KEY;
import static org.apache.atlas.repository.Constants.FULLTEXT_INDEX;
import static org.apache.atlas.repository.Constants.GUID_PROPERTY_KEY;
import static org.apache.atlas.repository.Constants.MODIFICATION_TIMESTAMP_PROPERTY_KEY;
import static org.apache.atlas.repository.Constants.MODIFIED_BY_KEY;
import static org.apache.atlas.repository.Constants.PROPAGATED_TRAIT_NAMES_PROPERTY_KEY;
import static org.apache.atlas.repository.Constants.RELATIONSHIP_GUID_PROPERTY_KEY;
import static org.apache.atlas.repository.Constants.STATE_PROPERTY_KEY;
import static org.apache.atlas.repository.Constants.SUPER_TYPES_PROPERTY_KEY;
import static org.apache.atlas.repository.Constants.TIMESTAMP_PROPERTY_KEY;
import static org.apache.atlas.repository.Constants.TRAIT_NAMES_PROPERTY_KEY;
import static org.apache.atlas.repository.Constants.TYPENAME_PROPERTY_KEY;
import static org.apache.atlas.repository.Constants.VERTEX_INDEX;
import static org.apache.atlas.repository.Constants.VERTEX_TYPE_PROPERTY_KEY;
import static org.apache.atlas.repository.Constants.*;
import static org.apache.atlas.repository.graphdb.AtlasCardinality.LIST;
import static org.apache.atlas.repository.graphdb.AtlasCardinality.SET;
import static org.apache.atlas.repository.graphdb.AtlasCardinality.SINGLE;
......@@ -101,6 +81,8 @@ public class GraphBackedSearchIndexer implements SearchIndexer, ActiveStateChang
private static final Logger LOG = LoggerFactory.getLogger(GraphBackedSearchIndexer.class);
private static final String VERTEX_ID_IN_IMPORT_KEY = "__vIdInImport";
private static final String EDGE_ID_IN_IMPORT_KEY = "__eIdInImport";
private static final List<Class> INDEX_EXCLUSION_CLASSES = new ArrayList() {
{
add(Boolean.class);
......@@ -284,6 +266,9 @@ public class GraphBackedSearchIndexer implements SearchIndexer, ActiveStateChang
createVertexIndex(management, PROPAGATED_TRAIT_NAMES_PROPERTY_KEY, String.class, false, LIST, true, true);
createVertexIndex(management, TYPENAME_PROPERTY_KEY, String.class, true, SINGLE, true, true);
createVertexIndex(management, VERTEX_TYPE_PROPERTY_KEY, String.class, false, SINGLE, true, true);
createVertexIndex(management, CLASSIFICATION_ENTITY_GUID, String.class, false, SINGLE, true, true);
createVertexIndex(management, VERTEX_ID_IN_IMPORT_KEY, Long.class, false, SINGLE, true, false);
// create vertex-centric index
createVertexCentricIndex(management, CLASSIFICATION_LABEL, AtlasEdgeDirection.BOTH, CLASSIFICATION_EDGE_NAME_PROPERTY_KEY, String.class, SINGLE);
......@@ -292,6 +277,7 @@ public class GraphBackedSearchIndexer implements SearchIndexer, ActiveStateChang
// create edge indexes
createEdgeIndex(management, RELATIONSHIP_GUID_PROPERTY_KEY, String.class, SINGLE, true);
createEdgeIndex(management, EDGE_ID_IN_IMPORT_KEY, String.class, SINGLE, true);
// create fulltext indexes
createFullTextIndex(management, ENTITY_TEXT_PROPERTY_KEY, String.class, SINGLE);
......
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.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.impexp;
import com.google.common.cache.CacheBuilder;
import com.google.common.cache.CacheLoader;
import com.google.common.cache.LoadingCache;
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.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);
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";
private final AtlasGraph graph;
private final MigrationStatus defaultStatus = new MigrationStatus();
private LoadingCache<String, MigrationStatus> cache;
@Inject
public MigrationProgressService(AtlasGraph graph) {
this.graph = graph;
}
public MigrationStatus getStatus() {
try {
if (cache == null) {
initCache();
cache.get(MIGRATION_STATUS_KEY);
}
if(cache.size() > 0) {
return cache.get(MIGRATION_STATUS_KEY);
}
return defaultStatus;
} catch (ExecutionException e) {
return defaultStatus;
}
}
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 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;
}
});
}
}
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS 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.common.annotations.VisibleForTesting;
import org.apache.atlas.exception.AtlasBaseException;
import org.apache.atlas.model.impexp.AtlasImportResult;
import org.apache.atlas.model.typedef.AtlasTypesDef;
import org.apache.atlas.repository.graph.GraphBackedSearchIndexer;
import org.apache.atlas.repository.impexp.ImportTypeDefProcessor;
import org.apache.atlas.repository.store.bootstrap.AtlasTypeDefStoreInitializer;
import org.apache.atlas.store.AtlasTypeDefStore;
import org.apache.atlas.type.AtlasType;
import org.apache.atlas.type.AtlasTypeRegistry;
import org.apache.commons.configuration.Configuration;
import org.apache.atlas.AtlasException;
import org.apache.atlas.service.Service;
import org.apache.commons.io.FileUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.springframework.stereotype.Component;
import javax.inject.Inject;
import java.io.File;
import java.io.FileInputStream;
import java.io.IOException;
import java.nio.file.Paths;
import static org.apache.atlas.AtlasConstants.ATLAS_MIGRATION_MODE_FILENAME;
@Component
public class DataMigrationService implements Service {
private static final Logger LOG = LoggerFactory.getLogger(DataMigrationService.class);
private static String ATLAS_MIGRATION_DATA_NAME = "atlas-migration-data.json";
private static String ATLAS_MIGRATION_TYPESDEF_NAME = "atlas-migration-typesdef.json";
private final Configuration configuration;
private final Thread thread;
@Inject
public DataMigrationService(AtlasTypeDefStore typeDefStore, Configuration configuration,
GraphBackedSearchIndexer indexer, AtlasTypeDefStoreInitializer storeInitializer,
AtlasTypeRegistry typeRegistry) {
this.configuration = configuration;
this.thread = new Thread(new FileImporter(typeDefStore, typeRegistry, storeInitializer, getFileName(), indexer));
}
@Override
public void start() {
Runtime.getRuntime().addShutdownHook(thread);
thread.start();
}
@Override
public void stop() {
try {
thread.join();
} catch (InterruptedException e) {
LOG.error("Data Migration: Interrupted", e);
}
}
public String getFileName() {
return configuration.getString(ATLAS_MIGRATION_MODE_FILENAME, "");
}
public static class FileImporter implements Runnable {
private final AtlasTypeDefStore typeDefStore;
private final String importDirectory;
private final GraphBackedSearchIndexer indexer;
private final AtlasTypeRegistry typeRegistry;
private final AtlasTypeDefStoreInitializer storeInitializer;
public FileImporter(AtlasTypeDefStore typeDefStore, AtlasTypeRegistry typeRegistry,
AtlasTypeDefStoreInitializer storeInitializer,
String directoryName, GraphBackedSearchIndexer indexer) {
this.typeDefStore = typeDefStore;
this.typeRegistry = typeRegistry;
this.storeInitializer = storeInitializer;
this.importDirectory = directoryName;
this.indexer = indexer;
}
public void performImport() throws AtlasBaseException {
try {
performInit();
FileInputStream fs = new FileInputStream(getFileFromImportDirectory(importDirectory, ATLAS_MIGRATION_DATA_NAME));
typeDefStore.loadLegacyData(RelationshipCacheGenerator.get(typeRegistry), fs);
} catch (Exception ex) {
LOG.error("Import failed!", ex);
throw new AtlasBaseException(ex);
}
}
private void performInit() throws AtlasBaseException, AtlasException {
storeInitializer.init();
processIncomingTypesDef(getFileFromImportDirectory(importDirectory, ATLAS_MIGRATION_TYPESDEF_NAME));
indexer.instanceIsActive();
}
@VisibleForTesting
void processIncomingTypesDef(File typesDefFile) throws AtlasBaseException {
try {
String jsonStr = FileUtils.readFileToString(typesDefFile);
AtlasTypesDef typesDef = AtlasType.fromJson(jsonStr, AtlasTypesDef.class);
ImportTypeDefProcessor processor = new ImportTypeDefProcessor(typeDefStore, typeRegistry);
processor.processTypes(typesDef, new AtlasImportResult());
} catch (IOException e) {
LOG.error("processIncomingTypesDef: Could not process file: {}! Imported data may not be usable.", typesDefFile.getName());
}
}
private File getFileFromImportDirectory(String importDirectory, String fileName) {
return Paths.get(importDirectory, fileName).toFile();
}
@Override
public void run() {
try {
performImport();
} catch (AtlasBaseException e) {
LOG.error("Data Migration:", e);
}
}
}
}
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS 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.typedef.AtlasRelationshipDef;
import org.apache.atlas.model.typedef.AtlasRelationshipEndDef;
import org.apache.atlas.repository.Constants;
import org.apache.atlas.type.AtlasEntityType;
import org.apache.atlas.type.AtlasRelationshipType;
import org.apache.atlas.type.AtlasTypeRegistry;
import org.apache.commons.lang.StringUtils;
import java.util.Collection;
import java.util.HashMap;
import java.util.Map;
public class RelationshipCacheGenerator {
public static Map<String, String> get(AtlasTypeRegistry typeRegistry) {
Map<String, String> ret = new HashMap<>();
Collection<AtlasRelationshipType> relationshipTypes = typeRegistry.getAllRelationshipTypes();
for (AtlasRelationshipType rt : relationshipTypes) {
AtlasRelationshipDef rd = rt.getRelationshipDef();
String relTypeName = rt.getTypeName();
add(ret, getKey(rd.getEndDef1(), rt.getEnd1Type()), relTypeName);
add(ret, getKey(rd.getEndDef2(), rt.getEnd2Type()), relTypeName);
}
return ret;
}
private static String getKey(AtlasRelationshipEndDef ed, AtlasEntityType rt) {
return getKey(ed.getIsLegacyAttribute(), rt.getTypeName(), ed.getName());
}
private static String getKey(String lhs, String rhs) {
return String.format("%s%s.%s", Constants.INTERNAL_PROPERTY_KEY_PREFIX, lhs, rhs);
}
private static String getKey(boolean isLegacy, String typeName, String name) {
if(!isLegacy) {
return "";
}
return getKey(typeName, name);
}
private static void add(Map<String, String> map, String key, String value) {
if(StringUtils.isEmpty(key) || map.containsKey(key)) {
return;
}
map.put(key, value);
}
}
......@@ -63,6 +63,7 @@ import java.util.Map;
import static com.fasterxml.jackson.annotation.JsonAutoDetect.Visibility.NONE;
import static com.fasterxml.jackson.annotation.JsonAutoDetect.Visibility.PUBLIC_ONLY;
import static org.apache.atlas.AtlasConstants.ATLAS_MIGRATION_MODE_FILENAME;
/**
......@@ -88,10 +89,10 @@ public class AtlasTypeDefStoreInitializer implements ActiveStateChangeHandler {
@PostConstruct
public void init() throws AtlasBaseException {
LOG.info("==> AtlasTypeDefStoreInitializer.init()");
boolean isMigrationEnabled = !StringUtils.isEmpty(conf.getString(ATLAS_MIGRATION_MODE_FILENAME));
if (!HAConfiguration.isHAEnabled(conf)) {
if (!HAConfiguration.isHAEnabled(conf) || isMigrationEnabled) {
atlasTypeDefStore.init();
loadBootstrapTypeDefs();
} else {
LOG.info("AtlasTypeDefStoreInitializer.init(): deferring type loading until instance activation");
......@@ -151,22 +152,34 @@ public class AtlasTypeDefStoreInitializer implements ActiveStateChangeHandler {
File[] typeDefFiles = typesDir.exists() ? typesDir.listFiles() : null;
if (typeDefFiles == null || typeDefFiles.length == 0) {
LOG.info("Types directory {} does not exist or not readable or has no typedef files", typesDirName );
LOG.info("Types directory {} does not exist or not readable or has no typedef files", typesDirName);
} else {
// sort the files by filename
Arrays.sort(typeDefFiles);
for (File typeDefFile : typeDefFiles) {
if (typeDefFile.isFile()) {
try {
readTypesFromFile(typeDefFile);
} catch (Throwable t) {
LOG.error("error while registering types in file {}", typeDefFile.getAbsolutePath(), t);
}
}
}
LOG.info("<== AtlasTypeDefStoreInitializer({})", typesDir);
}
public void readTypesFromFile(File typeDefFile) {
if (!typeDefFile.isFile()) {
return;
}
try {
String jsonStr = new String(Files.readAllBytes(typeDefFile.toPath()), StandardCharsets.UTF_8);
AtlasTypesDef typesDef = AtlasType.fromJson(jsonStr, AtlasTypesDef.class);
if (typesDef == null || typesDef.isEmpty()) {
LOG.info("No type in file {}", typeDefFile.getAbsolutePath());
continue;
return;
}
AtlasTypesDef typesToCreate = getTypesToCreate(typesDef, atlasTypeRegistry);
......@@ -184,12 +197,6 @@ public class AtlasTypeDefStoreInitializer implements ActiveStateChangeHandler {
LOG.error("error while registering types in file {}", typeDefFile.getAbsolutePath(), t);
}
}
}
applyTypePatches(typesDir.getPath());
}
LOG.info("<== AtlasTypeDefStoreInitializer({})", typesDir);
}
public static AtlasTypesDef getTypesToCreate(AtlasTypesDef typesDef, AtlasTypeRegistry typeRegistry) {
AtlasTypesDef typesToCreate = new AtlasTypesDef();
......
......@@ -43,6 +43,7 @@ import org.apache.atlas.utils.AtlasEntityUtil;
import org.apache.atlas.utils.AtlasPerfTracer;
import org.apache.commons.collections.CollectionUtils;
import org.apache.commons.collections.MapUtils;
import org.apache.commons.configuration.Configuration;
import org.apache.commons.lang3.StringUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
......@@ -51,6 +52,7 @@ import org.springframework.stereotype.Component;
import javax.inject.Inject;
import java.util.*;
import static org.apache.atlas.AtlasConstants.ATLAS_MIGRATION_MODE_FILENAME;
import static org.apache.atlas.model.instance.EntityMutations.EntityOperation.DELETE;
import static org.apache.atlas.model.instance.EntityMutations.EntityOperation.UPDATE;
......
......@@ -24,6 +24,7 @@ import static org.apache.atlas.repository.Constants.TYPE_CATEGORY_PROPERTY_KEY;
import static org.apache.atlas.repository.Constants.VERTEX_TYPE_PROPERTY_KEY;
import static org.apache.atlas.repository.store.graph.v1.AtlasGraphUtilsV1.VERTEX_TYPE;
import java.io.InputStream;
import java.util.Date;
import java.util.HashSet;
import java.util.Iterator;
......@@ -37,10 +38,7 @@ import org.apache.atlas.exception.AtlasBaseException;
import org.apache.atlas.listener.TypeDefChangeListener;
import org.apache.atlas.model.typedef.*;
import org.apache.atlas.repository.Constants;
import org.apache.atlas.repository.graphdb.AtlasEdge;
import org.apache.atlas.repository.graphdb.AtlasEdgeDirection;
import org.apache.atlas.repository.graphdb.AtlasGraph;
import org.apache.atlas.repository.graphdb.AtlasVertex;
import org.apache.atlas.repository.graphdb.*;
import org.apache.atlas.repository.store.graph.*;
import org.apache.atlas.type.AtlasType;
import org.apache.atlas.type.AtlasTypeRegistry;
......@@ -112,6 +110,11 @@ public class AtlasTypeDefGraphStoreV1 extends AtlasTypeDefGraphStore {
LOG.info("<== AtlasTypeDefGraphStoreV1.init()");
}
@Override
public void loadLegacyData(Map<String, String> relationshipCache, InputStream fs) throws AtlasBaseException {
getAtlasGraph().loadLegacyGraphSON(relationshipCache, fs);
}
AtlasGraph getAtlasGraph() { return atlasGraph; }
@VisibleForTesting
......@@ -275,9 +278,20 @@ public class AtlasTypeDefGraphStoreV1 extends AtlasTypeDefGraphStore {
String updatedBy = vertex.getProperty(Constants.MODIFIED_BY_KEY, String.class);
Long createTime = vertex.getProperty(Constants.TIMESTAMP_PROPERTY_KEY, Long.class);
Long updateTime = vertex.getProperty(Constants.MODIFICATION_TIMESTAMP_PROPERTY_KEY, Long.class);
Long version = vertex.getProperty(Constants.VERSION_PROPERTY_KEY, Long.class);
Object versionObj = vertex.getProperty(Constants.VERSION_PROPERTY_KEY, Object.class);
String options = vertex.getProperty(Constants.TYPEOPTIONS_PROPERTY_KEY, String.class);
Long version = null;
if(versionObj instanceof Number) {
version = ((Number)versionObj).longValue();
} else if (versionObj != null) {
version = Long.valueOf(versionObj.toString());
} else {
version = Long.valueOf(0);
}
typeDef.setName(name);
typeDef.setDescription(description);
typeDef.setTypeVersion(typeVersion);
......@@ -313,7 +327,15 @@ public class AtlasTypeDefGraphStoreV1 extends AtlasTypeDefGraphStore {
boolean ret = false;
if (isTypeVertex(vertex)) {
TypeCategory vertexCategory = vertex.getProperty(Constants.TYPE_CATEGORY_PROPERTY_KEY, TypeCategory.class);
Object objTypeCategory = vertex.getProperty(Constants.TYPE_CATEGORY_PROPERTY_KEY, Object.class);
TypeCategory vertexCategory = null;
if(objTypeCategory instanceof TypeCategory) {
vertexCategory = (TypeCategory) objTypeCategory;
} else if (objTypeCategory != null) {
vertexCategory = TypeCategory.valueOf(objTypeCategory.toString());
}
ret = category.equals(vertexCategory);
}
......
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS 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.AtlasGraphProvider;
import org.apache.atlas.repository.graph.GraphBackedSearchIndexer;
import org.apache.atlas.repository.graph.GraphHelper;
import org.apache.atlas.repository.graphdb.AtlasEdgeDirection;
import org.apache.atlas.repository.graphdb.AtlasGraph;
import org.apache.atlas.repository.graphdb.AtlasVertex;
import org.apache.atlas.repository.store.bootstrap.AtlasTypeDefStoreInitializer;
import org.apache.atlas.runner.LocalSolrRunner;
import org.apache.atlas.store.AtlasTypeDefStore;
import org.apache.atlas.type.AtlasTypeRegistry;
import org.apache.atlas.utils.TestResourceFileUtils;
import org.testng.annotations.AfterClass;
import org.testng.annotations.Guice;
import org.testng.annotations.Test;
import java.io.IOException;
import java.util.List;
import static org.apache.atlas.graph.GraphSandboxUtil.useLocalSolr;
import static org.apache.atlas.repository.impexp.ZipFileResourceTestUtils.loadModelFromJson;
import static org.testng.Assert.assertEquals;
@Guice(modules = TestModules.TestOnlyModule.class)
public class HiveParititionIT extends MigrationBaseAsserts {
@Inject
private AtlasTypeDefStore typeDefStore;
@Inject
private AtlasTypeRegistry typeRegistry;
@Inject
private AtlasTypeDefStoreInitializer storeInitializer;
@Inject
private GraphBackedSearchIndexer indexer;
@Inject
public HiveParititionIT(AtlasGraph graph) {
super(graph);
}
@AfterClass
public void clear() throws Exception {
AtlasGraphProvider.cleanup();
if (useLocalSolr()) {
LocalSolrRunner.stop();
}
}
@Test
public void fileImporterTest() throws IOException, AtlasBaseException {
loadModelFromJson("0000-Area0/0010-base_model.json", typeDefStore, typeRegistry);
loadModelFromJson("1000-Hadoop/1030-hive_model.json", typeDefStore, typeRegistry);
String directoryName = TestResourceFileUtils.getDirectory("parts_db");
DataMigrationService.FileImporter fi = new DataMigrationService.FileImporter(typeDefStore, typeRegistry,
storeInitializer, directoryName, indexer);
fi.run();
assertPartitionKeyProperty(getVertex("hive_table", "t1"), 1);
assertPartitionKeyProperty(getVertex("hive_table", "tv1"), 1);
assertHiveVertices(1, 2, 7);
assertTypeCountNameGuid("hive_db", 1, "parts_db", "ae30d78b-51b4-42ab-9436-8d60c8f68b95");
assertTypeCountNameGuid("hive_process", 1, "", "");
assertEdges("hive_db", "parts_db", AtlasEdgeDirection.IN,1, 1, "");
assertEdges("hive_table", "t1", AtlasEdgeDirection.OUT, 1, 1, "hive_db_tables");
assertEdges("hive_table", "tv1", AtlasEdgeDirection.OUT, 1, 1, "hive_db_tables");
assertMigrationStatus(136);
}
private void assertPartitionKeyProperty(AtlasVertex vertex, int expectedCount) {
List<String> keys = GraphHelper.getListProperty(vertex, "hive_table.partitionKeys");
assertEquals(keys.size(), expectedCount);
}
}
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.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.RequestContextV1;
import org.apache.atlas.TestModules;
import org.apache.atlas.TestUtilsV2;
import org.apache.atlas.exception.AtlasBaseException;
import org.apache.atlas.repository.graph.AtlasGraphProvider;
import org.apache.atlas.repository.graphdb.AtlasEdgeDirection;
import org.apache.atlas.repository.graphdb.AtlasGraph;
import org.apache.atlas.runner.LocalSolrRunner;
import org.apache.atlas.store.AtlasTypeDefStore;
import org.apache.atlas.type.AtlasTypeRegistry;
import org.apache.atlas.utils.TestResourceFileUtils;
import org.testng.ITestContext;
import org.testng.annotations.*;
import java.io.FileInputStream;
import java.io.IOException;
import static org.apache.atlas.graph.GraphSandboxUtil.useLocalSolr;
import static org.apache.atlas.repository.impexp.ZipFileResourceTestUtils.loadModelFromJson;
@Guice(modules = TestModules.TestOnlyModule.class)
public class HiveStocksIT extends MigrationBaseAsserts {
@Inject
private AtlasTypeDefStore typeDefStore;
@Inject
private AtlasTypeRegistry typeRegistry;
@Inject
public HiveStocksIT(AtlasGraph graph) {
super(graph);
}
@BeforeTest
public void setupTest() {
RequestContextV1.clear();
RequestContextV1.get().setUser(TestUtilsV2.TEST_USER, null);
}
@AfterClass
public void clear() throws Exception {
AtlasGraphProvider.cleanup();
if (useLocalSolr()) {
LocalSolrRunner.stop();
}
}
@DataProvider(name = "stocks-2-branch08-tag")
public static Object[][] getStocksTag(ITestContext context) throws IOException {
return new Object[][]{{ TestResourceFileUtils.getFileInputStream("stocks-2-0.8-extended-tag.json") }};
}
@Test(dataProvider = "stocks-2-branch08-tag")
public void migrateFromEarlierVersionWithTag(FileInputStream fs) throws AtlasBaseException, IOException {
loadModelFromJson("0000-Area0/0010-base_model.json", typeDefStore, typeRegistry);
loadModelFromJson("1000-Hadoop/1030-hive_model.json", typeDefStore, typeRegistry);
typeDefStore.loadLegacyData(RelationshipCacheGenerator.get(typeRegistry), fs);
assertHiveVertices(1, 1, 7);
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");
assertTypeCountNameGuid("hive_column", 1, "high", "d72ce4fb-6f17-4e68-aa85-967366c9e891");
assertTypeCountNameGuid("hive_column", 1, "open", "788ba8fe-b7d8-41ba-84ef-c929732924ec");
assertTypeCountNameGuid("hive_column", 1, "dt", "643a0a71-0d97-477d-a43b-7ca433f85160");
assertTypeCountNameGuid("hive_column", 1, "low", "38caeaf7-49e6-4d6d-8727-231406a46821");
assertTypeCountNameGuid("hive_column", 1, "close", "3bae9b76-f812-4745-b4d2-2a72d2773d07");
assertTypeCountNameGuid("hive_column", 1, "volume", "bee376a4-3d8d-4943-b7e8-9bce042c2657");
assertTypeCountNameGuid("hive_column", 1, "adj_close", "fcba2002-cb38-4c2e-b853-68d421d66703");
assertTypeCountNameGuid("hive_process", 0, "", "");
assertTypeCountNameGuid("hive_storagedesc", 1, "", "294290d8-4498-4677-973c-c266d594b039");
assertTypeCountNameGuid("Tag1", 1, "", "");
assertEdges(getVertex("hive_db", "stocks").getEdges(AtlasEdgeDirection.IN).iterator(),1, 1, "");
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(164);
}
}
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.atlas.repository.migration;
import org.apache.atlas.repository.graph.GraphHelper;
import org.apache.atlas.repository.graphdb.*;
import org.apache.commons.lang.StringUtils;
import java.util.Iterator;
import java.util.List;
import static org.testng.Assert.assertEquals;
import static org.testng.Assert.assertNotNull;
import static org.testng.AssertJUnit.assertTrue;
public class MigrationBaseAsserts {
protected static final String ASSERT_NAME_PROPERTY = "Asset.name";
private final String TYPE_NAME_PROPERTY = "__typeName";
private final String R_GUID_PROPERTY_NAME = "_r__guid";
protected AtlasGraph graph;
protected MigrationBaseAsserts(AtlasGraph graph) {
this.graph = graph;
}
protected void assertHiveVertices(int dbCount, int tableCount, int columnCount) {
int i = 0;
Iterator<AtlasVertex> results = getVertices("hive_db", null);
for (Iterator<AtlasVertex> it = results; it.hasNext(); i++) {
assertNotNull(it.next());
}
assertEquals(i, dbCount);
i = 0;
results = getVertices("hive_table", null);
for (Iterator<AtlasVertex> it = results; it.hasNext(); i++) {
assertNotNull(it.next());
}
assertEquals(i, tableCount);
i = 0;
results = getVertices("hive_column", null);
for (Iterator<AtlasVertex> it = results; it.hasNext(); i++) {
assertNotNull(it.next());
}
assertTrue(i > 0);
assertEquals(i, columnCount);
}
protected Iterator<AtlasVertex> getVertices(String typeName, String name) {
AtlasGraphQuery query = graph.query().has(TYPE_NAME_PROPERTY, typeName);
if(!StringUtils.isEmpty(name)) {
query = query.has(ASSERT_NAME_PROPERTY, name);
}
return query.vertices().iterator();
}
protected AtlasVertex getVertex(String typeName, String name) {
Iterator<AtlasVertex> iterator = getVertices(typeName, name);
return iterator.hasNext() ? iterator.next() : null;
}
protected void assertEdges(String typeName, String assetName, AtlasEdgeDirection edgeDirection, int startIdx, int expectedItems, String edgeTypeName) {
assertEdges(getVertex(typeName, assetName).getEdges(edgeDirection).iterator(),startIdx, expectedItems, edgeTypeName);
}
protected void assertEdges(Iterator<AtlasEdge> results, int startIdx, int expectedItems, String edgeTypeName) {
int count = 0;
AtlasEdge e = null;
for (Iterator<AtlasEdge> it = results; it.hasNext() && count < startIdx; count++) {
e = it.next();
}
assertNotNull(GraphHelper.getProperty(e, R_GUID_PROPERTY_NAME));
assertNotNull(GraphHelper.getProperty(e, "tagPropagation"));
if(StringUtils.isNotEmpty(edgeTypeName)) {
assertEquals(GraphHelper.getProperty(e, TYPE_NAME_PROPERTY), edgeTypeName, edgeTypeName);
}
assertEquals(count, expectedItems, String.format("%s", edgeTypeName));
}
protected void assertTypeAttribute(String typeName, int expectedSize, String name, String guid, String propertyName) {
AtlasVertex v = getVertex(typeName, name);
String guidActual = GraphHelper.getGuid(v);
List list = (List) GraphHelper.getProperty(v, propertyName);
assertEquals(guidActual, guid);
assertNotNull(list);
assertEquals(list.size(), expectedSize);
}
protected void assertTypeCountNameGuid(String typeName, int expectedItems, String name, String guid) {
Iterator<AtlasVertex> results = getVertices(typeName, name);
int count = 0;
for (Iterator<AtlasVertex> it = results; it.hasNext(); ) {
AtlasVertex v = it.next();
assertEquals(GraphHelper.getTypeName(v), typeName);
if(StringUtils.isNotEmpty(guid)) {
assertEquals(GraphHelper.getGuid(v), guid, name);
}
if(StringUtils.isNotEmpty(name)) {
assertEquals(GraphHelper.getProperty(v, ASSERT_NAME_PROPERTY), name, name);
}
count++;
}
assertEquals(count, expectedItems, String.format("%s:%s", typeName, name));
}
protected void assertMigrationStatus(int expectedTotalCount) {
AtlasVertex v = getVertex("__MigrationStatus", "");
assertEquals((long) GraphHelper.getProperty(v, "currentIndex"), expectedTotalCount);
}
}
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS 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.model.typedef.AtlasRelationshipDef;
import org.apache.atlas.repository.Constants;
import org.apache.atlas.repository.impexp.ZipFileResourceTestUtils;
import org.apache.atlas.repository.store.graph.AtlasEntityStore;
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.jcodings.util.Hash;
import org.testng.annotations.BeforeClass;
import org.testng.annotations.Guice;
import org.testng.annotations.Test;
import java.io.IOException;
import java.util.Collection;
import java.util.Map;
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.Assert.assertTrue;
@Guice(modules = TestModules.TestOnlyModule.class)
public class RelationshipMappingTest {
@Inject
private AtlasTypeDefStore typeDefStore;
@Inject
private AtlasTypeRegistry typeRegistry;
@BeforeClass
public void setup() throws IOException, AtlasBaseException {
loadModelFromJson("0000-Area0/0010-base_model.json", typeDefStore, typeRegistry);
loadModelFromJson("1000-Hadoop/1030-hive_model.json", typeDefStore, typeRegistry);
}
@Test
public void createLookup() {
Map<String, String> cache = RelationshipCacheGenerator.get(typeRegistry);
assertEquals(cache.size(), getLegacyAttributeCount() - 1);
for (Map.Entry<String, String> entry : cache.entrySet()) {
assertTrue(StringUtils.isNotEmpty(entry.getKey()));
assertTrue(entry.getKey().startsWith(Constants.INTERNAL_PROPERTY_KEY_PREFIX), entry.getKey());
}
}
private int getLegacyAttributeCount() {
int count = 0;
for (AtlasRelationshipType rt : typeRegistry.getAllRelationshipTypes()) {
AtlasRelationshipDef rd = rt.getRelationshipDef();
if(rd.getEndDef1().getIsLegacyAttribute()) {
count++;
}
if(rd.getEndDef2().getIsLegacyAttribute()) {
count++;
}
}
return count;
}
}
......@@ -54,6 +54,7 @@ import org.apache.atlas.type.AtlasTypeRegistry;
import org.apache.atlas.type.AtlasTypeUtil;
import org.apache.commons.collections.CollectionUtils;
import org.apache.commons.collections.MapUtils;
import org.apache.commons.configuration.Configuration;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.testng.Assert;
......@@ -107,6 +108,8 @@ public class AtlasEntityStoreV1Test {
@Inject
private EntityGraphMapper graphMapper;
@Inject
private Configuration configuration;
@BeforeClass
public void setUp() throws Exception {
......
......@@ -17,6 +17,7 @@
*/
package org.apache.atlas.repository.store.graph.v1;
import org.apache.commons.configuration.Configuration;
import com.google.common.collect.ImmutableList;
import org.apache.atlas.RequestContextV1;
import org.apache.atlas.TestModules;
......
......@@ -55,6 +55,11 @@ public class TestResourceFileUtils {
return fs;
}
public static String getDirectory(String subDir) {
final String userDir = System.getProperty("user.dir");
return getTestFilePath(userDir, subDir, "");
}
public static <T> T readObjectFromJson(String subDir, String filename, Class<T> objectClass) throws IOException {
final String userDir = System.getProperty("user.dir");
String filePath = getTestJsonPath(userDir, subDir, filename);
......
{"mode":"EXTENDED","vertices":[{"__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":"2acd11e7-9cbd-4d47-9ded-831dabf80206"},"__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":1522695099005},"__type":{"type":"string","value":"typeSystem"},"__type.description":{"type":"string","value":"hive_order"},"__timestamp":{"type":"long","value":1522695099005},"_id":16448,"_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":"8b3f3741-3ba9-49f1-b545-43f12fa6fe29"},"__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":1522695099132},"__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":1522695099132},"_id":32832,"_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":"14d28b0c-30b0-42e4-addd-52501ea461ed"},"__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":1522695099163},"__type":{"type":"string","value":"typeSystem"},"__type.description":{"type":"string","value":"hive_db"},"__timestamp":{"type":"long","value":1522695099163},"__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":49216,"_type":"vertex"},{"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 parts_db.t1.col4@cl1 name col4 position 0 type string table "},"Referenceable.qualifiedName":{"type":"string","value":"parts_db.t1.col4@cl1"},"__guid":{"type":"string","value":"aae38dd6-5632-48a4-9e41-cea65323e47b"},"__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":1522695186970},"Asset.owner":{"type":"string","value":"anonymous"},"hive_column.position":{"type":"integer","value":0},"__timestamp":{"type":"long","value":1522695179454},"_id":65600,"_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":"80dfb85b-bbde-4e1b-a194-cc69519123bc"},"__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":1522695094647},"__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":1522695094647},"_id":16520,"_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":"fb115e8e-818c-411b-9b93-4caf289f5413"},"__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":1522695099034},"__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":1522695099034},"_id":32904,"_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":"2e7ffda9-b958-4353-9183-22741089ea6d"},"__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":1522695100619},"__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":1522695099177},"_id":49288,"_type":"vertex"},{"Asset.name":{"type":"string","value":"col1"},"hive_column.type":{"type":"string","value":"int"},"__modifiedBy":{"type":"string","value":"anonymous"},"__state":{"type":"string","value":"ACTIVE"},"entityText":{"type":"string","value":"hive_column owner anonymous qualifiedName parts_db.tv1.col1@cl1 name col1 position 0 type int table "},"Referenceable.qualifiedName":{"type":"string","value":"parts_db.tv1.col1@cl1"},"__guid":{"type":"string","value":"fd4ef7cb-84ed-4068-b56e-efabcc83ae7c"},"__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":1522695186970},"Asset.owner":{"type":"string","value":"anonymous"},"hive_column.position":{"type":"integer","value":0},"__timestamp":{"type":"long","value":1522695183992},"_id":65672,"_type":"vertex"},{"Asset.name":{"type":"string","value":"col2"},"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 parts_db.tv1.col2@cl1 name col2 position 1 type string table "},"Referenceable.qualifiedName":{"type":"string","value":"parts_db.tv1.col2@cl1"},"__guid":{"type":"string","value":"33fc7cdf-fcee-461d-aa8f-c6b8f3ff0bb4"},"__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":1522695186970},"Asset.owner":{"type":"string","value":"anonymous"},"hive_column.position":{"type":"integer","value":1},"__timestamp":{"type":"long","value":1522695183992},"_id":82056,"_type":"vertex"},{"hive_storagedesc.inputFormat":{"type":"string","value":"org.apache.hadoop.mapred.SequenceFileInputFormat"},"__modifiedBy":{"type":"string","value":"anonymous"},"__state":{"type":"string","value":"ACTIVE"},"entityText":{"type":"string","value":"hive_storagedesc qualifiedName parts_db.tv1@cl1_storage storedAsSubDirectories false compressed false inputFormat org.apache.hadoop.mapred.SequenceFileInputFormat outputFormat org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat table serdeInfo serializationLib null name null parameters null numBuckets -1 "},"Referenceable.qualifiedName":{"type":"string","value":"parts_db.tv1@cl1_storage"},"__guid":{"type":"string","value":"02b6baa9-02c0-4ef6-a82c-a47b28bab91a"},"hive_storagedesc.storedAsSubDirectories":{"type":"boolean","value":false},"__version":{"type":"integer","value":0},"hive_storagedesc.outputFormat":{"type":"string","value":"org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat"},"hive_storagedesc.parameters":{"type":"list","value":[]},"__superTypeNames":{"type":"list","value":[{"type":"string","value":"Referenceable"}]},"__createdBy":{"type":"string","value":"anonymous"},"__typeName":{"type":"string","value":"hive_storagedesc"},"__modificationTimestamp":{"type":"long","value":1522695186970},"hive_storagedesc.compressed":{"type":"boolean","value":false},"hive_storagedesc.numBuckets":{"type":"integer","value":-1},"__timestamp":{"type":"long","value":1522695183992},"_id":98440,"_type":"vertex"},{"hive_serde.parameters":{"type":"list","value":[]},"__modifiedBy":{"type":"string","value":"anonymous"},"__state":{"type":"string","value":"ACTIVE"},"__createdBy":{"type":"string","value":"anonymous"},"__typeName":{"type":"string","value":"hive_serde"},"__modificationTimestamp":{"type":"long","value":1522695186970},"__timestamp":{"type":"long","value":1522695183992},"_id":114824,"_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":1522695095354},"__type":{"type":"string","value":"typeSystem"},"__type.options":{"type":"string","value":"null"},"__type.description":{"type":"string","value":"DataSet"},"__guid":{"type":"string","value":"8eb418e7-6f18-44eb-a0d0-d762e753b071"},"__timestamp":{"type":"long","value":1522695095354},"_id":16568,"_type":"vertex"},{"Asset.name":{"type":"string","value":"col2"},"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 parts_db.t1.col2@cl1 name col2 position 1 type string table "},"Referenceable.qualifiedName":{"type":"string","value":"parts_db.t1.col2@cl1"},"__guid":{"type":"string","value":"4a56254e-eb4b-4d8c-bbdf-bdb7a81a4b18"},"__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":1522695186970},"Asset.owner":{"type":"string","value":"anonymous"},"hive_column.position":{"type":"integer","value":1},"__timestamp":{"type":"long","value":1522695179454},"_id":32952,"_type":"vertex"},{"hive_storagedesc.location":{"type":"string","value":"hdfs://localhost.localdomain:8020/apps/hive/warehouse/parts_db.db/t1"},"hive_storagedesc.inputFormat":{"type":"string","value":"org.apache.hadoop.mapred.TextInputFormat"},"__modifiedBy":{"type":"string","value":"anonymous"},"__state":{"type":"string","value":"ACTIVE"},"entityText":{"type":"string","value":"hive_storagedesc qualifiedName parts_db.t1@cl1_storage storedAsSubDirectories false location hdfs://localhost.localdomain:8020/apps/hive/warehouse/parts_db.db/t1 compressed false inputFormat org.apache.hadoop.mapred.TextInputFormat outputFormat org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat table serdeInfo serializationLib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe name null parameters serialization.format 1 numBuckets -1 "},"Referenceable.qualifiedName":{"type":"string","value":"parts_db.t1@cl1_storage"},"__guid":{"type":"string","value":"793900ed-16eb-436e-ad3f-dae391b3c09e"},"hive_storagedesc.storedAsSubDirectories":{"type":"boolean","value":false},"__version":{"type":"integer","value":0},"hive_storagedesc.outputFormat":{"type":"string","value":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat"},"hive_storagedesc.parameters":{"type":"list","value":[]},"__superTypeNames":{"type":"list","value":[{"type":"string","value":"Referenceable"}]},"__createdBy":{"type":"string","value":"anonymous"},"__typeName":{"type":"string","value":"hive_storagedesc"},"__modificationTimestamp":{"type":"long","value":1522695186970},"hive_storagedesc.compressed":{"type":"boolean","value":false},"hive_storagedesc.numBuckets":{"type":"integer","value":-1},"__timestamp":{"type":"long","value":1522695179454},"_id":49336,"_type":"vertex"},{"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 parts_db.tv1.col4@cl1 name col4 position 0 type string table "},"Referenceable.qualifiedName":{"type":"string","value":"parts_db.tv1.col4@cl1"},"__guid":{"type":"string","value":"615a5604-ac47-45b0-9ec3-73b705b5c619"},"__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":1522695186970},"Asset.owner":{"type":"string","value":"anonymous"},"hive_column.position":{"type":"integer","value":0},"__timestamp":{"type":"long","value":1522695183992},"_id":65720,"_type":"vertex"},{"hive_column_lineage.depenendencyType":{"type":"string","value":"SIMPLE"},"Asset.name":{"type":"string","value":"create view tv1 partitioned on (col4) as select col1, col2, col4 from t1:col2"},"Process.inputs":{"type":"list","value":[{"type":"string","value":"4fmf-1rco-2uqd-pfc"}]},"Process.outputs":{"type":"list","value":[{"type":"string","value":"4avr-1rco-2wb9-1rbc"}]},"__modifiedBy":{"type":"string","value":"anonymous"},"__state":{"type":"string","value":"ACTIVE"},"entityText":{"type":"string","value":"hive_column_lineage outputs qualifiedName parts_db.tv1@cl1:1522695183000:col2 inputs query name create view tv1 partitioned on (col4) as select col1, col2, col4 from t1:col2 depenendencyType SIMPLE "},"Referenceable.qualifiedName":{"type":"string","value":"parts_db.tv1@cl1:1522695183000:col2"},"__guid":{"type":"string","value":"872073fc-fd7a-4587-9f9c-9de1be8444d5"},"__version":{"type":"integer","value":0},"__superTypeNames":{"type":"list","value":[{"type":"string","value":"Asset"},{"type":"string","value":"Process"},{"type":"string","value":"Referenceable"}]},"__createdBy":{"type":"string","value":"anonymous"},"__typeName":{"type":"string","value":"hive_column_lineage"},"__modificationTimestamp":{"type":"long","value":1522695186459},"__timestamp":{"type":"long","value":1522695186459},"_id":82104,"_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":"2b9a6b23-c108-4a8d-ac12-483ce314a846"},"__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":1522695093946},"__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":1522695093946},"_id":16592,"_type":"vertex"},{"Asset.name":{"type":"string","value":"col1"},"hive_column.type":{"type":"string","value":"int"},"__modifiedBy":{"type":"string","value":"anonymous"},"__state":{"type":"string","value":"ACTIVE"},"entityText":{"type":"string","value":"hive_column owner anonymous qualifiedName parts_db.t1.col1@cl1 name col1 position 0 type int table "},"Referenceable.qualifiedName":{"type":"string","value":"parts_db.t1.col1@cl1"},"__guid":{"type":"string","value":"d1a16e97-7171-4119-bfc1-8a66e36ddfb1"},"__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":1522695186970},"Asset.owner":{"type":"string","value":"anonymous"},"hive_column.position":{"type":"integer","value":0},"__timestamp":{"type":"long","value":1522695179454},"_id":32976,"_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":"7ad1392b-05bb-4919-be2b-861b6a165f76"},"__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":1522695096763},"__type":{"type":"string","value":"typeSystem"},"__type.description":{"type":"string","value":"__AtlasUserSavedSearch"},"__timestamp":{"type":"long","value":1522695096763},"_id":16616,"_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":"252ac491-054a-4074-b636-cf7024553ead"},"__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":1522695098299},"__type":{"type":"string","value":"typeSystem"},"__type.description":{"type":"string","value":"hive_principal_type"},"__timestamp":{"type":"long","value":1522695098299},"_id":16664,"_type":"vertex"},{"hive_table.createTime":{"type":"long","value":1522695183000},"hive_table.tableType":{"type":"string","value":"VIRTUAL_VIEW"},"Asset.name":{"type":"string","value":"tv1"},"__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:53:03 PDT 2018 qualifiedName parts_db.tv1@cl1 columns viewExpandedText select `t1`.`col1`, `t1`.`col2`, `t1`.`col4` from `parts_db`.`t1` sd tableType VIRTUAL_VIEW createTime Mon Apr 02 11:53:03 PDT 2018 name tv1 partitionKeys parameters transient_lastDdlTime 1522695183 db retention 0 viewOriginalText select col1, col2, col4 from t1 "},"Referenceable.qualifiedName":{"type":"string","value":"parts_db.tv1@cl1"},"hive_table.parameters.transient_lastDdlTime":{"type":"string","value":"1522695183"},"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":"1ueb-pi0-7q51-1epk"}]},"__guid":{"type":"string","value":"3f248307-5307-4c01-b12e-776c3f410448"},"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 `t1`.`col1`, `t1`.`col2`, `t1`.`col4` from `parts_db`.`t1`"},"__createdBy":{"type":"string","value":"anonymous"},"__typeName":{"type":"string","value":"hive_table"},"__modificationTimestamp":{"type":"long","value":1522695186970},"Asset.owner":{"type":"string","value":"anonymous"},"hive_table.lastAccessTime":{"type":"long","value":1522695183000},"hive_table.viewOriginalText":{"type":"string","value":"select col1, col2, col4 from t1"},"hive_table.columns":{"type":"list","value":[{"type":"string","value":"1hr7-pi0-7rpx-1eo8"},{"type":"string","value":"1jc3-pi0-7rpx-1rbc"}]},"__timestamp":{"type":"long","value":1522695183992},"_id":33048,"_type":"vertex"},{"hive_db.parameters":{"type":"list","value":[]},"Asset.name":{"type":"string","value":"parts_db"},"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 parts_db@cl1 clusterName cl1 name parts_db location hdfs://localhost.localdomain:8020/apps/hive/warehouse/parts_db.db "},"Referenceable.qualifiedName":{"type":"string","value":"parts_db@cl1"},"__guid":{"type":"string","value":"ae30d78b-51b4-42ab-9436-8d60c8f68b95"},"__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":1522695186970},"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/parts_db.db"},"__timestamp":{"type":"long","value":1522695166550},"_id":16672,"_type":"vertex"},{"Asset.name":{"type":"string","value":"col3"},"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 parts_db.t1.col3@cl1 name col3 position 2 type string table "},"Referenceable.qualifiedName":{"type":"string","value":"parts_db.t1.col3@cl1"},"__guid":{"type":"string","value":"fbc6012f-ddd5-4626-a132-8da1c8fbca54"},"__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":1522695186970},"Asset.owner":{"type":"string","value":"anonymous"},"hive_column.position":{"type":"integer","value":2},"__timestamp":{"type":"long","value":1522695179454},"_id":33056,"_type":"vertex"},{"hive_column_lineage.depenendencyType":{"type":"string","value":"SIMPLE"},"Asset.name":{"type":"string","value":"create view tv1 partitioned on (col4) as select col1, col2, col4 from t1:col1"},"Process.inputs":{"type":"list","value":[{"type":"string","value":"3fac-125c-2uqd-pg0"}]},"Process.outputs":{"type":"list","value":[{"type":"string","value":"3ajo-125c-2wb9-1eo8"}]},"__modifiedBy":{"type":"string","value":"anonymous"},"__state":{"type":"string","value":"ACTIVE"},"entityText":{"type":"string","value":"hive_column_lineage outputs qualifiedName parts_db.tv1@cl1:1522695183000:col1 inputs query name create view tv1 partitioned on (col4) as select col1, col2, col4 from t1:col1 depenendencyType SIMPLE "},"Referenceable.qualifiedName":{"type":"string","value":"parts_db.tv1@cl1:1522695183000:col1"},"__guid":{"type":"string","value":"7d689c57-7b36-45c1-8052-e19c673736ec"},"__version":{"type":"integer","value":0},"__superTypeNames":{"type":"list","value":[{"type":"string","value":"Asset"},{"type":"string","value":"Process"},{"type":"string","value":"Referenceable"}]},"__createdBy":{"type":"string","value":"anonymous"},"__typeName":{"type":"string","value":"hive_column_lineage"},"__modificationTimestamp":{"type":"long","value":1522695186970},"__timestamp":{"type":"long","value":1522695186970},"_id":49440,"_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":"c98d432c-b263-4ff0-a0ec-0ed5bec2d5c4"},"__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":1522695093163},"__type":{"type":"string","value":"typeSystem"},"__type.description":{"type":"string","value":"TaxonomyTerm"},"__timestamp":{"type":"long","value":1522695093163},"_id":16784,"_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":1522695093960},"__type":{"type":"string","value":"typeSystem"},"__type.options":{"type":"string","value":"null"},"__type.description":{"type":"string","value":"__internal"},"__guid":{"type":"string","value":"1d67b415-d737-44d6-a017-41b5fdf032aa"},"__timestamp":{"type":"long","value":1522695093960},"_id":33168,"_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":1522695095359},"__type":{"type":"string","value":"typeSystem"},"__type.options":{"type":"string","value":"null"},"__type.description":{"type":"string","value":"Infrastructure"},"__guid":{"type":"string","value":"7debb9e2-47a3-45ce-9d96-db6d91aad5d9"},"__timestamp":{"type":"long","value":1522695095359},"_id":49552,"_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":"e31c5978-6c25-48a7-8aa5-093c6b5ba5ef"},"__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":1522695095364},"__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":1522695095364},"_id":65936,"_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":"088a8b06-c0f9-4b49-9dc4-fe02affcb609"},"__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":1522695099017},"__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":1522695099017},"_id":82320,"_type":"vertex"},{"hive_table.createTime":{"type":"long","value":1522695179000},"hive_table.tableType":{"type":"string","value":"MANAGED_TABLE"},"Asset.name":{"type":"string","value":"t1"},"__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:52:59 PDT 2018 qualifiedName parts_db.t1@cl1 columns sd tableType MANAGED_TABLE createTime Mon Apr 02 11:52:59 PDT 2018 name t1 partitionKeys parameters transient_lastDdlTime 1522695179 db retention 0 "},"Referenceable.qualifiedName":{"type":"string","value":"parts_db.t1@cl1"},"hive_table.parameters.transient_lastDdlTime":{"type":"string","value":"1522695179"},"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":"4lyq-245s-7q51-1em8"}]},"__guid":{"type":"string","value":"808b29b8-9ce0-4f86-aadd-639d3401bccb"},"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"}]},"__createdBy":{"type":"string","value":"anonymous"},"__typeName":{"type":"string","value":"hive_table"},"__modificationTimestamp":{"type":"long","value":1522695186970},"Asset.owner":{"type":"string","value":"anonymous"},"hive_table.lastAccessTime":{"type":"long","value":1522695179000},"hive_table.columns":{"type":"list","value":[{"type":"string","value":"49bm-245s-7rpx-pg0"},{"type":"string","value":"4awi-245s-7rpx-pfc"},{"type":"string","value":"4che-245s-7rpx-pi8"}]},"__timestamp":{"type":"long","value":1522695179454},"_id":98704,"_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":"881822d1-efd1-4dc6-aea4-03d88b13f989"},"__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":1522695096058},"__type":{"type":"string","value":"typeSystem"},"__type.description":{"type":"string","value":"__AtlasUserProfile"},"__timestamp":{"type":"long","value":1522695096058},"_id":16808,"_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":1522695099075},"__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":"abe70487-b34e-4b54-9e08-2426e958d376"},"__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":1522695100458},"__type.description":{"type":"string","value":"hive_table"},"_id":33192,"_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":"78572253-4763-405b-bbc9-0b77626535fe"},"__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":1522695099192},"__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":1522695099192},"_id":49576,"_type":"vertex"},{"hive_serde.parameters.serialization.format":{"type":"string","value":"1"},"hive_serde.parameters":{"type":"list","value":[{"type":"string","value":"serialization.format"}]},"__modifiedBy":{"type":"string","value":"anonymous"},"__state":{"type":"string","value":"ACTIVE"},"__createdBy":{"type":"string","value":"anonymous"},"__typeName":{"type":"string","value":"hive_serde"},"__modificationTimestamp":{"type":"long","value":1522695186970},"hive_serde.serializationLib":{"type":"string","value":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe"},"__timestamp":{"type":"long","value":1522695179454},"_id":65960,"_type":"vertex"},{"Asset.name":{"type":"string","value":"create view tv1 partitioned on (col4) as select col1, col2, col4 from t1"},"Process.inputs":{"type":"list","value":[{"type":"string","value":"4sad-1rjc-2uqd-245s"}]},"Process.outputs":{"type":"list","value":[{"type":"string","value":"4lyt-1rjc-2wb9-pi0"}]},"__modifiedBy":{"type":"string","value":"anonymous"},"__state":{"type":"string","value":"ACTIVE"},"hive_process.queryPlan":{"type":"string","value":"Not Supported"},"hive_process.queryText":{"type":"string","value":"create view tv1 partitioned on (col4) as select col1, col2, col4 from t1"},"entityText":{"type":"string","value":"hive_process outputs recentQueries create view tv1 partitioned on (col4) as select col1, col2, col4 from t1 qualifiedName parts_db.tv1@cl1:1522695183000 inputs userName anonymous queryId hive_20180402115303_03c23526-e1ee-4cf9-80af-6c8652bcefa3 clusterName cl1 name create view tv1 partitioned on (col4) as select col1, col2, col4 from t1 queryText create view tv1 partitioned on (col4) as select col1, col2, col4 from t1 startTime Mon Apr 02 11:53:03 PDT 2018 operationType CREATEVIEW queryPlan Not Supported endTime Mon Apr 02 11:53:03 PDT 2018 "},"Referenceable.qualifiedName":{"type":"string","value":"parts_db.tv1@cl1:1522695183000"},"__guid":{"type":"string","value":"330938ef-2f16-43c5-b251-a72455775440"},"hive_process.operationType":{"type":"string","value":"CREATEVIEW"},"hive_process.endTime":{"type":"long","value":1522695183616},"__version":{"type":"integer","value":0},"hive_process.recentQueries":{"type":"list","value":[{"type":"string","value":"create view tv1 partitioned on (col4) as select col1, col2, col4 from t1"}]},"hive_process.startTime":{"type":"long","value":1522695183339},"hive_process.clusterName":{"type":"string","value":"cl1"},"__superTypeNames":{"type":"list","value":[{"type":"string","value":"Asset"},{"type":"string","value":"Process"},{"type":"string","value":"Referenceable"}]},"__createdBy":{"type":"string","value":"anonymous"},"__typeName":{"type":"string","value":"hive_process"},"hive_process.queryId":{"type":"string","value":"hive_20180402115303_03c23526-e1ee-4cf9-80af-6c8652bcefa3"},"__modificationTimestamp":{"type":"long","value":1522695186970},"hive_process.userName":{"type":"string","value":"anonymous"},"__timestamp":{"type":"long","value":1522695185696},"_id":82344,"_type":"vertex"}],"edges":[{"_id":"2rk8-pc0-2711-csw","_type":"edge","_outV":32832,"_inV":16592,"_label":"__type..supertype"},{"_id":"2mtk-pc0-6gb9-pm0","_type":"edge","_outV":32832,"_inV":33192,"_label":"__type.edge.hive_storagedesc.table"},{"_id":"2oeg-pc0-6hw5-1rio","_type":"edge","_outV":32832,"_inV":82320,"_label":"__type.edge.hive_storagedesc.serdeInfo"},{"_id":"2pzc-pc0-6jh1-cow","_type":"edge","_outV":32832,"_inV":16448,"_label":"__type.edge.hive_storagedesc.sortCols"},{"_id":"2waw-11z4-2711-cqw","_type":"edge","_outV":49216,"_inV":16520,"_label":"__type..supertype"},{"_id":"2uq0-11z4-2711-csw","_type":"edge","_outV":49216,"_inV":16592,"_label":"__type..supertype"},{"_id":"2t54-11z4-6l1x-cuw","_type":"edge","_outV":49216,"_inV":16664,"_label":"__type.edge.hive_db.ownerType"},{"__modifiedBy":{"type":"string","value":"anonymous"},"__state":{"type":"string","value":"ACTIVE"},"__createdBy":{"type":"string","value":"anonymous"},"__modificationTimestamp":{"type":"long","value":1522695179454},"__timestamp":{"type":"long","value":1522695179454},"_id":"3oqw-1em8-8tn9-245s","_type":"edge","_outV":65600,"_inV":98704,"_label":"__hive_column.table"},{"_id":"2l8x-pe0-2711-1evk","_type":"edge","_outV":32904,"_inV":65936,"_label":"__type..supertype"},{"_id":"2oep-1214-2711-cs8","_type":"edge","_outV":49288,"_inV":16568,"_label":"__type..supertype"},{"_id":"2mtt-1214-6mmt-pm0","_type":"edge","_outV":49288,"_inV":33192,"_label":"__type.edge.hive_column.table"},{"__modifiedBy":{"type":"string","value":"anonymous"},"__state":{"type":"string","value":"ACTIVE"},"__createdBy":{"type":"string","value":"anonymous"},"__modificationTimestamp":{"type":"long","value":1522695183992},"__timestamp":{"type":"long","value":1522695183992},"_id":"56hd-1eo8-8tn9-pi0","_type":"edge","_outV":65672,"_inV":33048,"_label":"__hive_column.table"},{"__modifiedBy":{"type":"string","value":"anonymous"},"__state":{"type":"string","value":"ACTIVE"},"__createdBy":{"type":"string","value":"anonymous"},"__modificationTimestamp":{"type":"long","value":1522695183992},"__timestamp":{"type":"long","value":1522695183992},"_id":"5fyp-1rbc-8tn9-pi0","_type":"edge","_outV":82056,"_inV":33048,"_label":"__hive_column.table"},{"__modifiedBy":{"type":"string","value":"anonymous"},"__state":{"type":"string","value":"ACTIVE"},"__createdBy":{"type":"string","value":"anonymous"},"__modificationTimestamp":{"type":"long","value":1522695183992},"__timestamp":{"type":"long","value":1522695183992},"_id":"5r0x-23yg-82s5-pi0","_type":"edge","_outV":98440,"_inV":33048,"_label":"__hive_storagedesc.table"},{"__modifiedBy":{"type":"string","value":"anonymous"},"__state":{"type":"string","value":"ACTIVE"},"__createdBy":{"type":"string","value":"anonymous"},"__modificationTimestamp":{"type":"long","value":1522695183992},"__timestamp":{"type":"long","value":1522695183992},"_id":"658x-23yg-9frp-2glk","_type":"edge","_outV":98440,"_inV":114824,"_label":"__hive_storagedesc.serdeInfo"},{"_id":"npz-cs8-2711-cqw","_type":"edge","_outV":16568,"_inV":16520,"_label":"__type..supertype"},{"_id":"m53-cs8-2711-csw","_type":"edge","_outV":16568,"_inV":16592,"_label":"__type..supertype"},{"__modifiedBy":{"type":"string","value":"anonymous"},"__state":{"type":"string","value":"ACTIVE"},"__createdBy":{"type":"string","value":"anonymous"},"__modificationTimestamp":{"type":"long","value":1522695179454},"__timestamp":{"type":"long","value":1522695179454},"_id":"1vyv-pfc-8tn9-245s","_type":"edge","_outV":32952,"_inV":98704,"_label":"__hive_column.table"},{"__modifiedBy":{"type":"string","value":"anonymous"},"__state":{"type":"string","value":"ACTIVE"},"__createdBy":{"type":"string","value":"anonymous"},"__modificationTimestamp":{"type":"long","value":1522695179454},"__timestamp":{"type":"long","value":1522695179454},"_id":"28lz-122g-82s5-245s","_type":"edge","_outV":49336,"_inV":98704,"_label":"__hive_storagedesc.table"},{"__modifiedBy":{"type":"string","value":"anonymous"},"__state":{"type":"string","value":"ACTIVE"},"__createdBy":{"type":"string","value":"anonymous"},"__modificationTimestamp":{"type":"long","value":1522695179454},"__timestamp":{"type":"long","value":1522695179454},"_id":"2a6v-122g-9frp-1ew8","_type":"edge","_outV":49336,"_inV":65960,"_label":"__hive_storagedesc.serdeInfo"},{"__modifiedBy":{"type":"string","value":"anonymous"},"__state":{"type":"string","value":"ACTIVE"},"__createdBy":{"type":"string","value":"anonymous"},"__modificationTimestamp":{"type":"long","value":1522695183992},"__timestamp":{"type":"long","value":1522695183992},"_id":"3c47-1epk-8tn9-pi0","_type":"edge","_outV":65720,"_inV":33048,"_label":"__hive_column.table"},{"__modifiedBy":{"type":"string","value":"anonymous"},"__state":{"type":"string","value":"ACTIVE"},"__createdBy":{"type":"string","value":"anonymous"},"__modificationTimestamp":{"type":"long","value":1522695186459},"__timestamp":{"type":"long","value":1522695186459},"_id":"4fmf-1rco-2uqd-pfc","_type":"edge","_outV":82104,"_inV":32952,"_label":"__Process.inputs"},{"__modifiedBy":{"type":"string","value":"anonymous"},"__state":{"type":"string","value":"ACTIVE"},"__createdBy":{"type":"string","value":"anonymous"},"__modificationTimestamp":{"type":"long","value":1522695186459},"__timestamp":{"type":"long","value":1522695186459},"_id":"4avr-1rco-2wb9-1rbc","_type":"edge","_outV":82104,"_inV":82056,"_label":"__Process.outputs"},{"__modifiedBy":{"type":"string","value":"anonymous"},"__state":{"type":"string","value":"ACTIVE"},"__createdBy":{"type":"string","value":"anonymous"},"__modificationTimestamp":{"type":"long","value":1522695186459},"__timestamp":{"type":"long","value":1522695186459},"_id":"4is7-1rco-8v85-1rjc","_type":"edge","_outV":82104,"_inV":82344,"_label":"__hive_column_lineage.query"},{"__modifiedBy":{"type":"string","value":"anonymous"},"__state":{"type":"string","value":"ACTIVE"},"__createdBy":{"type":"string","value":"anonymous"},"__modificationTimestamp":{"type":"long","value":1522695179454},"__timestamp":{"type":"long","value":1522695179454},"_id":"1g62-pg0-8tn9-245s","_type":"edge","_outV":32976,"_inV":98704,"_label":"__hive_column.table"},{"_id":"vml-ctk-2711-plc","_type":"edge","_outV":16616,"_inV":33168,"_label":"__type..supertype"},{"__modifiedBy":{"type":"string","value":"anonymous"},"__state":{"type":"string","value":"ACTIVE"},"__createdBy":{"type":"string","value":"anonymous"},"__modificationTimestamp":{"type":"long","value":1522695183992},"__timestamp":{"type":"long","value":1522695183992},"_id":"20pv-pi0-7gnp-cv4","_type":"edge","_outV":33048,"_inV":16672,"_label":"__hive_table.db"},{"__modifiedBy":{"type":"string","value":"anonymous"},"__state":{"type":"string","value":"ACTIVE"},"__createdBy":{"type":"string","value":"anonymous"},"__modificationTimestamp":{"type":"long","value":1522695183992},"__timestamp":{"type":"long","value":1522695183992},"_id":"1o2r-pi0-7ok5-23yg","_type":"edge","_outV":33048,"_inV":98440,"_label":"__hive_table.sd"},{"__modifiedBy":{"type":"string","value":"anonymous"},"__state":{"type":"string","value":"ACTIVE"},"__createdBy":{"type":"string","value":"anonymous"},"__modificationTimestamp":{"type":"long","value":1522695183992},"__timestamp":{"type":"long","value":1522695183992},"_id":"1ueb-pi0-7q51-1epk","_type":"edge","_outV":33048,"_inV":65720,"_label":"__hive_table.partitionKeys"},{"__modifiedBy":{"type":"string","value":"anonymous"},"__state":{"type":"string","value":"ACTIVE"},"__createdBy":{"type":"string","value":"anonymous"},"__modificationTimestamp":{"type":"long","value":1522695183992},"__timestamp":{"type":"long","value":1522695183992},"_id":"1hr7-pi0-7rpx-1eo8","_type":"edge","_outV":33048,"_inV":65672,"_label":"__hive_table.columns"},{"__modifiedBy":{"type":"string","value":"anonymous"},"__state":{"type":"string","value":"ACTIVE"},"__createdBy":{"type":"string","value":"anonymous"},"__modificationTimestamp":{"type":"long","value":1522695183992},"__timestamp":{"type":"long","value":1522695183992},"_id":"1jc3-pi0-7rpx-1rbc","_type":"edge","_outV":33048,"_inV":82056,"_label":"__hive_table.columns"},{"__modifiedBy":{"type":"string","value":"anonymous"},"__state":{"type":"string","value":"ACTIVE"},"__createdBy":{"type":"string","value":"anonymous"},"__modificationTimestamp":{"type":"long","value":1522695179454},"__timestamp":{"type":"long","value":1522695179454},"_id":"1mhw-pi8-8tn9-245s","_type":"edge","_outV":33056,"_inV":98704,"_label":"__hive_column.table"},{"__modifiedBy":{"type":"string","value":"anonymous"},"__state":{"type":"string","value":"ACTIVE"},"__createdBy":{"type":"string","value":"anonymous"},"__modificationTimestamp":{"type":"long","value":1522695186970},"__timestamp":{"type":"long","value":1522695186970},"_id":"3fac-125c-2uqd-pg0","_type":"edge","_outV":49440,"_inV":32976,"_label":"__Process.inputs"},{"__modifiedBy":{"type":"string","value":"anonymous"},"__state":{"type":"string","value":"ACTIVE"},"__createdBy":{"type":"string","value":"anonymous"},"__modificationTimestamp":{"type":"long","value":1522695186970},"__timestamp":{"type":"long","value":1522695186970},"_id":"3ajo-125c-2wb9-1eo8","_type":"edge","_outV":49440,"_inV":65672,"_label":"__Process.outputs"},{"__modifiedBy":{"type":"string","value":"anonymous"},"__state":{"type":"string","value":"ACTIVE"},"__createdBy":{"type":"string","value":"anonymous"},"__modificationTimestamp":{"type":"long","value":1522695186970},"__timestamp":{"type":"long","value":1522695186970},"_id":"3ig4-125c-8v85-1rjc","_type":"edge","_outV":49440,"_inV":82344,"_label":"__hive_column_lineage.query"},{"_id":"2l9u-128g-2711-cqw","_type":"edge","_outV":49552,"_inV":16520,"_label":"__type..supertype"},{"_id":"2joy-128g-2711-csw","_type":"edge","_outV":49552,"_inV":16592,"_label":"__type..supertype"},{"_id":"2rle-1evk-2711-cqw","_type":"edge","_outV":65936,"_inV":16520,"_label":"__type..supertype"},{"_id":"2q0i-1evk-2711-csw","_type":"edge","_outV":65936,"_inV":16592,"_label":"__type..supertype"},{"_id":"2muq-1evk-28lx-cs8","_type":"edge","_outV":65936,"_inV":16568,"_label":"__type.edge.Process.inputs"},{"_id":"2ofm-1evk-2a6t-cs8","_type":"edge","_outV":65936,"_inV":16568,"_label":"__type.edge.Process.outputs"},{"__modifiedBy":{"type":"string","value":"anonymous"},"__state":{"type":"string","value":"ACTIVE"},"__createdBy":{"type":"string","value":"anonymous"},"__modificationTimestamp":{"type":"long","value":1522695179454},"__timestamp":{"type":"long","value":1522695179454},"_id":"4saa-245s-7gnp-cv4","_type":"edge","_outV":98704,"_inV":16672,"_label":"__hive_table.db"},{"__modifiedBy":{"type":"string","value":"anonymous"},"__state":{"type":"string","value":"ACTIVE"},"__createdBy":{"type":"string","value":"anonymous"},"__modificationTimestamp":{"type":"long","value":1522695179454},"__timestamp":{"type":"long","value":1522695179454},"_id":"4fn6-245s-7ok5-122g","_type":"edge","_outV":98704,"_inV":49336,"_label":"__hive_table.sd"},{"__modifiedBy":{"type":"string","value":"anonymous"},"__state":{"type":"string","value":"ACTIVE"},"__createdBy":{"type":"string","value":"anonymous"},"__modificationTimestamp":{"type":"long","value":1522695179454},"__timestamp":{"type":"long","value":1522695179454},"_id":"4lyq-245s-7q51-1em8","_type":"edge","_outV":98704,"_inV":65600,"_label":"__hive_table.partitionKeys"},{"__modifiedBy":{"type":"string","value":"anonymous"},"__state":{"type":"string","value":"ACTIVE"},"__createdBy":{"type":"string","value":"anonymous"},"__modificationTimestamp":{"type":"long","value":1522695179454},"__timestamp":{"type":"long","value":1522695179454},"_id":"4awi-245s-7rpx-pfc","_type":"edge","_outV":98704,"_inV":32952,"_label":"__hive_table.columns"},{"__modifiedBy":{"type":"string","value":"anonymous"},"__state":{"type":"string","value":"ACTIVE"},"__createdBy":{"type":"string","value":"anonymous"},"__modificationTimestamp":{"type":"long","value":1522695179454},"__timestamp":{"type":"long","value":1522695179454},"_id":"49bm-245s-7rpx-pg0","_type":"edge","_outV":98704,"_inV":32976,"_label":"__hive_table.columns"},{"__modifiedBy":{"type":"string","value":"anonymous"},"__state":{"type":"string","value":"ACTIVE"},"__createdBy":{"type":"string","value":"anonymous"},"__modificationTimestamp":{"type":"long","value":1522695179454},"__timestamp":{"type":"long","value":1522695179454},"_id":"4che-245s-7rpx-pi8","_type":"edge","_outV":98704,"_inV":33056,"_label":"__hive_table.columns"},{"_id":"u2d-cyw-2711-plc","_type":"edge","_outV":16808,"_inV":33168,"_label":"__type..supertype"},{"_id":"shh-cyw-2brp-ctk","_type":"edge","_outV":16808,"_inV":16616,"_label":"__type.edge.__AtlasUserProfile.savedSearches"},{"_id":"312t-pm0-2711-cs8","_type":"edge","_outV":33192,"_inV":16568,"_label":"__type..supertype"},{"_id":"2ur9-pm0-69zp-11z4","_type":"edge","_outV":33192,"_inV":49216,"_label":"__type.edge.hive_table.db"},{"_id":"2wc5-pm0-6bkl-pc0","_type":"edge","_outV":33192,"_inV":32832,"_label":"__type.edge.hive_table.sd"},{"_id":"2xx1-pm0-6d5h-1214","_type":"edge","_outV":33192,"_inV":49288,"_label":"__type.edge.hive_table.partitionKeys"},{"_id":"2zhx-pm0-6eqd-1214","_type":"edge","_outV":33192,"_inV":49288,"_label":"__type.edge.hive_table.columns"},{"_id":"348l-1294-2711-1evk","_type":"edge","_outV":49576,"_inV":65936,"_label":"__type..supertype"},{"_id":"32np-1294-6o7p-pe0","_type":"edge","_outV":49576,"_inV":32904,"_label":"__type.edge.hive_column_lineage.query"},{"__modifiedBy":{"type":"string","value":"anonymous"},"__state":{"type":"string","value":"ACTIVE"},"__createdBy":{"type":"string","value":"anonymous"},"__modificationTimestamp":{"type":"long","value":1522695185696},"__timestamp":{"type":"long","value":1522695185696},"_id":"4sad-1rjc-2uqd-245s","_type":"edge","_outV":82344,"_inV":98704,"_label":"__Process.inputs"},{"__modifiedBy":{"type":"string","value":"anonymous"},"__state":{"type":"string","value":"ACTIVE"},"__createdBy":{"type":"string","value":"anonymous"},"__modificationTimestamp":{"type":"long","value":1522695185696},"__timestamp":{"type":"long","value":1522695185696},"_id":"4lyt-1rjc-2wb9-pi0","_type":"edge","_outV":82344,"_inV":33048,"_label":"__Process.outputs"}]}
\ No newline at end of file
{"enumDefs":[{"category":"ENUM","guid":"252ac491-054a-4074-b636-cf7024553ead","createdBy":"root","updatedBy":"root","createTime":1522695098299,"updateTime":1522695098299,"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}]}],"structDefs":[{"category":"STRUCT","guid":"2acd11e7-9cbd-4d47-9ded-831dabf80206","createdBy":"root","updatedBy":"root","createTime":1522695099005,"updateTime":1522695099005,"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":"088a8b06-c0f9-4b49-9dc4-fe02affcb609","createdBy":"root","updatedBy":"root","createTime":1522695099017,"updateTime":1522695099017,"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}]}],"classificationDefs":[{"category":"CLASSIFICATION","guid":"c98d432c-b263-4ff0-a0ec-0ed5bec2d5c4","createdBy":"root","updatedBy":"root","createTime":1522695093163,"updateTime":1522695093163,"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":"8eb418e7-6f18-44eb-a0d0-d762e753b071","createdBy":"root","updatedBy":"root","createTime":1522695095354,"updateTime":1522695095354,"version":1,"name":"DataSet","description":"DataSet","typeVersion":"1.0","attributeDefs":[],"superTypes":["Referenceable","Asset"]},{"category":"ENTITY","guid":"7ad1392b-05bb-4919-be2b-861b6a165f76","createdBy":"root","updatedBy":"root","createTime":1522695096763,"updateTime":1522695096763,"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":"e31c5978-6c25-48a7-8aa5-093c6b5ba5ef","createdBy":"root","updatedBy":"root","createTime":1522695095364,"updateTime":1522695095364,"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":"abe70487-b34e-4b54-9e08-2426e958d376","createdBy":"root","updatedBy":"root","createTime":1522695099075,"updateTime":1522695100458,"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":"881822d1-efd1-4dc6-aea4-03d88b13f989","createdBy":"root","updatedBy":"root","createTime":1522695096058,"updateTime":1522695096058,"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":"14d28b0c-30b0-42e4-addd-52501ea461ed","createdBy":"root","updatedBy":"root","createTime":1522695099163,"updateTime":1522695099163,"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":"fb115e8e-818c-411b-9b93-4caf289f5413","createdBy":"root","updatedBy":"root","createTime":1522695099034,"updateTime":1522695099034,"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":"8b3f3741-3ba9-49f1-b545-43f12fa6fe29","createdBy":"root","updatedBy":"root","createTime":1522695099132,"updateTime":1522695099132,"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":"2b9a6b23-c108-4a8d-ac12-483ce314a846","createdBy":"root","updatedBy":"root","createTime":1522695093946,"updateTime":1522695093946,"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":"78572253-4763-405b-bbc9-0b77626535fe","createdBy":"root","updatedBy":"root","createTime":1522695099192,"updateTime":1522695099192,"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":"7debb9e2-47a3-45ce-9d96-db6d91aad5d9","createdBy":"root","updatedBy":"root","createTime":1522695095359,"updateTime":1522695095359,"version":1,"name":"Infrastructure","description":"Infrastructure","typeVersion":"1.0","attributeDefs":[],"superTypes":["Referenceable","Asset"]},{"category":"ENTITY","guid":"1d67b415-d737-44d6-a017-41b5fdf032aa","createdBy":"root","updatedBy":"root","createTime":1522695093960,"updateTime":1522695093960,"version":1,"name":"__internal","description":"__internal","typeVersion":"1.0","attributeDefs":[],"superTypes":[]},{"category":"ENTITY","guid":"80dfb85b-bbde-4e1b-a194-cc69519123bc","createdBy":"root","updatedBy":"root","createTime":1522695094647,"updateTime":1522695094647,"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":"2e7ffda9-b958-4353-9183-22741089ea6d","createdBy":"root","updatedBy":"root","createTime":1522695099177,"updateTime":1522695100619,"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"]}]}
\ No newline at end of file
This source diff could not be displayed because it is too large. You can view the blob instead.
......@@ -87,6 +87,18 @@ public class ActiveServerFilter implements Filter {
HttpServletResponse httpServletResponse = (HttpServletResponse) servletResponse;
LOG.error("Instance in transition. Service may not be ready to return a result");
httpServletResponse.sendError(HttpServletResponse.SC_SERVICE_UNAVAILABLE);
} if(serviceState.isInstanceInMigration()) {
HttpServletResponse httpServletResponse = (HttpServletResponse) servletResponse;
LOG.error("Instance in migration. Service may not be ready to return a result");
httpServletResponse.sendError(HttpServletResponse.SC_SERVICE_UNAVAILABLE);
} if (serviceState.isInstanceInMigration()) {
HttpServletResponse httpServletResponse = (HttpServletResponse) servletResponse;
LOG.error("Instance in migration. Service may not be ready to return a result");
httpServletResponse.sendError(HttpServletResponse.SC_SERVICE_UNAVAILABLE);
} else if (serviceState.isInstanceInMigration()) {
HttpServletResponse httpServletResponse = (HttpServletResponse) servletResponse;
LOG.error("Instance in migration. Service may not be ready to return a result");
httpServletResponse.sendError(HttpServletResponse.SC_SERVICE_UNAVAILABLE);
} else {
HttpServletResponse httpServletResponse = (HttpServletResponse) servletResponse;
String activeServerAddress = activeInstanceState.getActiveServerAddress();
......
......@@ -28,15 +28,9 @@ import org.apache.atlas.authorize.AtlasPrivilege;
import org.apache.atlas.authorize.AtlasAuthorizationUtils;
import org.apache.atlas.discovery.SearchContext;
import org.apache.atlas.exception.AtlasBaseException;
import org.apache.atlas.model.impexp.AtlasExportRequest;
import org.apache.atlas.model.impexp.AtlasExportResult;
import org.apache.atlas.model.impexp.AtlasImportRequest;
import org.apache.atlas.model.impexp.AtlasImportResult;
import org.apache.atlas.model.impexp.*;
import org.apache.atlas.model.metrics.AtlasMetrics;
import org.apache.atlas.repository.impexp.ExportService;
import org.apache.atlas.repository.impexp.ImportService;
import org.apache.atlas.repository.impexp.ZipSink;
import org.apache.atlas.repository.impexp.ZipSource;
import org.apache.atlas.repository.impexp.*;
import org.apache.atlas.services.MetricsService;
import org.apache.atlas.type.AtlasType;
import org.apache.atlas.type.AtlasTypeRegistry;
......@@ -75,17 +69,12 @@ import javax.ws.rs.core.MediaType;
import javax.ws.rs.core.Response;
import java.io.IOException;
import java.io.InputStream;
import java.util.Collection;
import java.util.Collections;
import java.util.HashSet;
import java.util.HashMap;
import java.util.Map;
import java.util.Set;
import java.util.*;
import java.util.concurrent.locks.ReentrantLock;
/**
* Jersey Resource for admin operations.
* Jersey Resource for admin operations
*/
@Path("admin")
@Singleton
......@@ -93,16 +82,6 @@ import java.util.concurrent.locks.ReentrantLock;
public class AdminResource {
private static final Logger LOG = LoggerFactory.getLogger(AdminResource.class);
@Context
private HttpServletRequest httpServletRequest;
@Context
private HttpServletResponse httpServletResponse;
private final AtlasTypeRegistry typeRegistry;
private final ReentrantLock importExportOperationLock;
private static final String isCSRF_ENABLED = "atlas.rest-csrf.enabled";
private static final String BROWSER_USER_AGENT_PARAM = "atlas.rest-csrf.browser-useragents-regex";
private static final String CUSTOM_METHODS_TO_IGNORE_PARAM = "atlas.rest-csrf.methods-to-ignore";
......@@ -111,6 +90,13 @@ public class AdminResource {
private static final String isEntityCreateAllowed = "atlas.entity.create.allowed";
private static final String editableEntityTypes = "atlas.ui.editable.entity.types";
private static final String DEFAULT_EDITABLE_ENTITY_TYPES = "hdfs_path,hbase_table,hbase_column,hbase_column_family,kafka_topic,hbase_namespace";
@Context
private HttpServletRequest httpServletRequest;
@Context
private HttpServletResponse httpServletResponse;
private Response version;
private final ServiceState serviceState;
......@@ -119,6 +105,9 @@ public class AdminResource {
private final ExportService exportService;
private final ImportService importService;
private final SearchTracker activeSearches;
private final AtlasTypeRegistry typeRegistry;
private final MigrationProgressService migrationProgressService;
private final ReentrantLock importExportOperationLock;
static {
try {
......@@ -129,15 +118,16 @@ public class AdminResource {
}
@Inject
public AdminResource(ServiceState serviceState, MetricsService metricsService,
ExportService exportService, ImportService importService,
SearchTracker activeSearches, AtlasTypeRegistry typeRegistry) {
public AdminResource(ServiceState serviceState, MetricsService metricsService, AtlasTypeRegistry typeRegistry,
ExportService exportService, ImportService importService, SearchTracker activeSearches,
MigrationProgressService migrationProgressService) {
this.serviceState = serviceState;
this.metricsService = metricsService;
this.exportService = exportService;
this.importService = importService;
this.activeSearches = activeSearches;
this.typeRegistry = typeRegistry;
this.migrationProgressService = migrationProgressService;
importExportOperationLock = new ReentrantLock();
}
......@@ -223,7 +213,17 @@ public class AdminResource {
LOG.debug("==> AdminResource.getStatus()");
}
Map<String, Object> responseData = Collections.singletonMap(AtlasClient.STATUS, serviceState.getState().toString());
Map<String, Object> responseData = new HashMap() {{
put(AtlasClient.STATUS, serviceState.getState().toString());
}};
if(serviceState.isInstanceInMigration()) {
MigrationStatus status = migrationProgressService.getStatus();
if (status != null) {
responseData.put("MigrationStatus", status);
}
}
Response response = Response.ok(AtlasJson.toV1Json(responseData)).build();
if (LOG.isDebugEnabled()) {
......
......@@ -24,6 +24,7 @@ import org.apache.atlas.web.filters.AtlasCSRFPreventionFilter;
import org.apache.atlas.web.filters.AtlasKnoxSSOAuthenticationFilter;
import org.apache.atlas.web.filters.StaleTransactionCleanupFilter;
import org.apache.commons.configuration.Configuration;
import org.apache.commons.lang.StringUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.springframework.security.config.annotation.authentication.builders.AuthenticationManagerBuilder;
......@@ -44,6 +45,8 @@ import org.springframework.security.web.util.matcher.RequestMatcher;
import javax.inject.Inject;
import java.util.LinkedHashMap;
import static org.apache.atlas.AtlasConstants.ATLAS_MIGRATION_MODE_FILENAME;
@EnableWebSecurity
@EnableGlobalMethodSecurity(prePostEnabled = true)
public class AtlasSecurityConfig extends WebSecurityConfigurerAdapter {
......@@ -151,8 +154,14 @@ public class AtlasSecurityConfig extends WebSecurityConfigurerAdapter {
//@formatter:on
if (configuration.getBoolean("atlas.server.ha.enabled", false)) {
boolean configMigrationEnabled = !StringUtils.isEmpty(configuration.getString(ATLAS_MIGRATION_MODE_FILENAME));
if (configuration.getBoolean("atlas.server.ha.enabled", false) ||
configMigrationEnabled) {
if(configMigrationEnabled) {
LOG.info("Atlas is in Migration Mode, enabling ActiveServerFilter");
} else {
LOG.info("Atlas is in HA Mode, enabling ActiveServerFilter");
}
httpSecurity.addFilterAfter(activeServerFilter, BasicAuthenticationFilter.class);
}
httpSecurity
......
......@@ -23,12 +23,15 @@ import org.apache.atlas.ApplicationProperties;
import org.apache.atlas.AtlasException;
import org.apache.atlas.ha.HAConfiguration;
import org.apache.commons.configuration.Configuration;
import org.apache.commons.lang.StringUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.springframework.stereotype.Component;
import javax.inject.Singleton;
import static org.apache.atlas.AtlasConstants.ATLAS_MIGRATION_MODE_FILENAME;
/**
* A class that maintains the state of this instance.
*
......@@ -38,14 +41,14 @@ import javax.inject.Singleton;
@Singleton
@Component
public class ServiceState {
private static final Logger LOG = LoggerFactory.getLogger(ServiceState.class);
public enum ServiceStateValue {
ACTIVE,
PASSIVE,
BECOMING_ACTIVE,
BECOMING_PASSIVE
BECOMING_PASSIVE,
MIGRATING
}
private Configuration configuration;
......@@ -57,8 +60,16 @@ public class ServiceState {
public ServiceState(Configuration configuration) {
this.configuration = configuration;
state = !HAConfiguration.isHAEnabled(configuration) ?
ServiceStateValue.ACTIVE : ServiceStateValue.PASSIVE;
if(StringUtils.isNotEmpty(configuration.getString(ATLAS_MIGRATION_MODE_FILENAME))) {
state = ServiceStateValue.MIGRATING;
} else {
state = !HAConfiguration.isHAEnabled(configuration) ? ServiceStateValue.ACTIVE : ServiceStateValue.PASSIVE;
}
if(!StringUtils.isEmpty(configuration.getString(ATLAS_MIGRATION_MODE_FILENAME, ""))) {
state = ServiceStateValue.MIGRATING;
}
}
public ServiceStateValue getState() {
......@@ -96,4 +107,13 @@ public class ServiceState {
return state == ServiceStateValue.BECOMING_ACTIVE
|| state == ServiceStateValue.BECOMING_PASSIVE;
}
public void setMigration() {
LOG.warn("Instance in {}", state);
setState(ServiceStateValue.MIGRATING);
}
public boolean isInstanceInMigration() {
return getState() == ServiceStateValue.MIGRATING;
}
}
......@@ -51,7 +51,7 @@ public class AdminResourceTest {
when(serviceState.getState()).thenReturn(ServiceState.ServiceStateValue.ACTIVE);
AdminResource adminResource = new AdminResource(serviceState, null, null, null, null, null);
AdminResource adminResource = new AdminResource(serviceState, null, null, null, null, null, null);
Response response = adminResource.getStatus();
assertEquals(response.getStatus(), HttpServletResponse.SC_OK);
JsonNode entity = AtlasJson.parseToV1JsonNode((String) response.getEntity());
......@@ -62,7 +62,7 @@ public class AdminResourceTest {
public void testResourceGetsValueFromServiceState() throws IOException {
when(serviceState.getState()).thenReturn(ServiceState.ServiceStateValue.PASSIVE);
AdminResource adminResource = new AdminResource(serviceState, null, null, null, null, null);
AdminResource adminResource = new AdminResource(serviceState, null, null, null, null, null, null);
Response response = adminResource.getStatus();
verify(serviceState).getState();
......
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