Skip to content

Commit

Permalink
[CARBONDATA-2484][LUCENE]Refactor distributable code and lauch job to…
Browse files Browse the repository at this point in the history
… clear the datamap from executor(clears segmentMap and remove datamap from cache)

Problem:
During query, blockletDataMapFactory maintains a segmentMap which has mapping of
segmentId -> list of index file, and this will be used while getting the extended blocklet
by checking whether the blocklet present in the index or not.
In case of Lucene, the datamap job will be launched and during pruning the segmentMap will be added
in executor and this map will be cleared in driver when drop table is called, but it will not be cleared in executor.
so when the query is fired after table or datamap is dropped, the lucene query fails.

Solution:
So when drop table or drop datamap is called a job is launched which clears the datamaps from segmentMap and cache and
then clears in driver.

This PR also refactors the datamap job classes and other common classes

This closes #2310
  • Loading branch information
akashrn5 authored and ravipesala committed May 18, 2018
1 parent 2f79e14 commit 2018048
Show file tree
Hide file tree
Showing 33 changed files with 357 additions and 122 deletions.
Expand Up @@ -15,7 +15,7 @@
* limitations under the License.
*/

package org.apache.carbondata.hadoop.api;
package org.apache.carbondata.core.datamap;

import java.util.List;

Expand Down
Expand Up @@ -130,6 +130,25 @@ private DataMapExprWrapper chooseDataMap(DataMapLevel level, FilterResolverIntf
return null;
}

/**
* Get all datamaps of the table for clearing purpose
*/
public DataMapExprWrapper getAllDataMapsForClear(CarbonTable carbonTable)
throws IOException {
List<TableDataMap> allDataMapFG =
DataMapStoreManager.getInstance().getAllDataMap(carbonTable);
DataMapExprWrapper initialExpr = null;
if (allDataMapFG.size() > 0) {
initialExpr = new DataMapExprWrapperImpl(allDataMapFG.get(0), null);

for (int i = 1; i < allDataMapFG.size(); i++) {
initialExpr = new AndDataMapExprWrapper(initialExpr,
new DataMapExprWrapperImpl(allDataMapFG.get(i), null), null);
}
}
return initialExpr;
}

/**
* Returns default blocklet datamap
* @param carbonTable
Expand Down
Expand Up @@ -14,7 +14,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.carbondata.hadoop.api;
package org.apache.carbondata.core.datamap;

import java.io.Serializable;
import java.util.List;
Expand Down
Expand Up @@ -36,6 +36,7 @@
import org.apache.carbondata.core.indexstore.SegmentPropertiesFetcher;
import org.apache.carbondata.core.indexstore.blockletindex.BlockletDataMapFactory;
import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
import org.apache.carbondata.core.metadata.CarbonMetadata;
import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
import org.apache.carbondata.core.metadata.schema.table.DataMapSchema;
import org.apache.carbondata.core.metadata.schema.table.DataMapSchemaStorageProvider;
Expand All @@ -57,6 +58,10 @@ public final class DataMapStoreManager {

private static DataMapStoreManager instance = new DataMapStoreManager();

public Map<String, List<TableDataMap>> getAllDataMaps() {
return allDataMaps;
}

/**
* Contains the list of datamaps for each table.
*/
Expand Down Expand Up @@ -364,17 +369,58 @@ public void clearInvalidSegments(CarbonTable carbonTable, List<Segment> segments
* @param identifier Table identifier
*/
public void clearDataMaps(AbsoluteTableIdentifier identifier) {
CarbonTable carbonTable = getCarbonTable(identifier);
String tableUniqueName = identifier.getCarbonTableIdentifier().getTableUniqueName();
List<TableDataMap> tableIndices = allDataMaps.get(tableUniqueName);
if (null != carbonTable && tableIndices != null) {
try {
DataMapUtil.executeDataMapJobForClearingDataMaps(carbonTable);
} catch (IOException e) {
LOGGER.error(e, "clear dataMap job failed");
// ignoring the exception
}
}
segmentRefreshMap.remove(identifier.uniqueName());
clearDataMaps(tableUniqueName);
allDataMaps.remove(tableUniqueName);
}

/**
* This method returns the carbonTable from identifier
* @param identifier
* @return
*/
public CarbonTable getCarbonTable(AbsoluteTableIdentifier identifier) {
CarbonTable carbonTable = null;
carbonTable = CarbonMetadata.getInstance()
.getCarbonTable(identifier.getDatabaseName(), identifier.getTableName());
if (carbonTable == null) {
try {
carbonTable = CarbonTable
.buildFromTablePath(identifier.getTableName(), identifier.getDatabaseName(),
identifier.getTablePath());
} catch (IOException e) {
LOGGER.error("failed to get carbon table from table Path");
// ignoring exception
}
}
return carbonTable;
}

/**
* this methos clears the datamap of table from memory
*/
public void clearDataMaps(String tableUniqName) {
List<TableDataMap> tableIndices = allDataMaps.get(tableUniqName);
if (tableIndices != null) {
for (TableDataMap tableDataMap : tableIndices) {
if (tableDataMap != null) {
// clear the segmentMap in BlockletDetailsFetcher,else the Segment will remain in executor
// and the query fails as we will check whether the blocklet contains in the index or not
tableDataMap.getBlockletDetailsFetcher().clear();
tableDataMap.clear();
break;
}
}
allDataMaps.remove(tableUniqueName);
}
}

Expand All @@ -384,14 +430,21 @@ public void clearDataMaps(AbsoluteTableIdentifier identifier) {
* @param identifier Table identifier
*/
public void clearDataMap(AbsoluteTableIdentifier identifier, String dataMapName) {
List<TableDataMap> tableIndices =
allDataMaps.get(identifier.getCarbonTableIdentifier().getTableUniqueName());
CarbonTable carbonTable = getCarbonTable(identifier);
String tableUniqueName = identifier.getCarbonTableIdentifier().getTableUniqueName();
List<TableDataMap> tableIndices = allDataMaps.get(tableUniqueName);
if (tableIndices != null) {
int i = 0;
for (TableDataMap tableDataMap : tableIndices) {
if (tableDataMap != null && dataMapName
if (carbonTable != null && tableDataMap != null && dataMapName
.equalsIgnoreCase(tableDataMap.getDataMapSchema().getDataMapName())) {
tableDataMap.clear();
try {
DataMapUtil.executeDataMapJobForClearingDataMaps(carbonTable);
tableDataMap.clear();
} catch (IOException e) {
LOGGER.error(e, "clear dataMap job failed");
// ignoring the exception
}
tableDataMap.deleteDatamapData();
tableIndices.remove(i);
break;
Expand Down
157 changes: 157 additions & 0 deletions core/src/main/java/org/apache/carbondata/core/datamap/DataMapUtil.java
@@ -0,0 +1,157 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package org.apache.carbondata.core.datamap;

import java.io.IOException;
import java.lang.reflect.Constructor;
import java.util.List;

import org.apache.carbondata.common.logging.LogService;
import org.apache.carbondata.common.logging.LogServiceFactory;
import org.apache.carbondata.core.datamap.dev.expr.DataMapExprWrapper;
import org.apache.carbondata.core.indexstore.ExtendedBlocklet;
import org.apache.carbondata.core.indexstore.PartitionSpec;
import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
import org.apache.carbondata.core.scan.filter.resolver.FilterResolverIntf;
import org.apache.carbondata.core.statusmanager.SegmentStatusManager;
import org.apache.carbondata.core.util.ObjectSerializationUtil;

import org.apache.hadoop.conf.Configuration;

public class DataMapUtil {

private static final String DATA_MAP_DSTR = "mapreduce.input.carboninputformat.datamapdstr";

private static final LogService LOGGER =
LogServiceFactory.getLogService(DataMapUtil.class.getName());

/**
* Creates instance for the DataMap Job class
*
* @param className
* @return
*/
public static Object createDataMapJob(String className) {
try {
return Class.forName(className).getDeclaredConstructors()[0].newInstance();
} catch (Exception e) {
LOGGER.error(e);
return null;
}
}

/**
* This method sets the datamapJob in the configuration
* @param configuration
* @param dataMapJob
* @throws IOException
*/
public static void setDataMapJob(Configuration configuration, Object dataMapJob)
throws IOException {
if (dataMapJob != null) {
String toString = ObjectSerializationUtil.convertObjectToString(dataMapJob);
configuration.set(DATA_MAP_DSTR, toString);
}
}

/**
* get datamap job from the configuration
* @param configuration job configuration
* @return DataMap Job
* @throws IOException
*/
public static DataMapJob getDataMapJob(Configuration configuration) throws IOException {
String jobString = configuration.get(DATA_MAP_DSTR);
if (jobString != null) {
return (DataMapJob) ObjectSerializationUtil.convertStringToObject(jobString);
}
return null;
}

/**
* This method gets the datamapJob and call execute , this job will be launched before clearing
* datamaps from driver side during drop table and drop datamap and clears the datamap in executor
* side
* @param carbonTable
* @throws IOException
*/
public static void executeDataMapJobForClearingDataMaps(CarbonTable carbonTable)
throws IOException {
String dataMapJobClassName = "org.apache.carbondata.spark.rdd.SparkDataMapJob";
DataMapJob dataMapJob = (DataMapJob) createDataMapJob(dataMapJobClassName);
String className = "org.apache.carbondata.core.datamap.DistributableDataMapFormat";
SegmentStatusManager.ValidAndInvalidSegmentsInfo validAndInvalidSegmentsInfo =
getValidAndInvalidSegments(carbonTable);
List<Segment> validSegments = validAndInvalidSegmentsInfo.getValidSegments();
List<Segment> invalidSegments = validAndInvalidSegmentsInfo.getInvalidSegments();
DataMapExprWrapper dataMapExprWrapper = null;
if (DataMapStoreManager.getInstance().getAllDataMap(carbonTable).size() > 0) {
DataMapChooser dataMapChooser = new DataMapChooser(carbonTable);
dataMapExprWrapper = dataMapChooser.getAllDataMapsForClear(carbonTable);
} else {
return;
}
DistributableDataMapFormat dataMapFormat =
createDataMapJob(carbonTable, dataMapExprWrapper, validSegments, invalidSegments, null,
className, true);
dataMapJob.execute(dataMapFormat, null);
}

private static DistributableDataMapFormat createDataMapJob(CarbonTable carbonTable,
DataMapExprWrapper dataMapExprWrapper, List<Segment> validsegments,
List<Segment> invalidSegments, List<PartitionSpec> partitionsToPrune, String clsName,
boolean isJobToClearDataMaps) {
try {
Constructor<?> cons = Class.forName(clsName).getDeclaredConstructors()[0];
return (DistributableDataMapFormat) cons
.newInstance(carbonTable, dataMapExprWrapper, validsegments, invalidSegments,
partitionsToPrune, isJobToClearDataMaps);
} catch (Exception e) {
throw new RuntimeException(e);
}
}

/**
* this method gets the datamapJob and call execute of that job, this will be launched for
* distributed CG or FG
* @return list of Extended blocklets after pruning
*/
public static List<ExtendedBlocklet> executeDataMapJob(CarbonTable carbonTable,
FilterResolverIntf resolver, List<Segment> validSegments,
DataMapExprWrapper dataMapExprWrapper, DataMapJob dataMapJob,
List<PartitionSpec> partitionsToPrune) throws IOException {
String className = "org.apache.carbondata.core.datamap.DistributableDataMapFormat";
SegmentStatusManager.ValidAndInvalidSegmentsInfo validAndInvalidSegmentsInfo =
getValidAndInvalidSegments(carbonTable);
List<Segment> invalidSegments = validAndInvalidSegmentsInfo.getInvalidSegments();
DistributableDataMapFormat dataMapFormat =
createDataMapJob(carbonTable, dataMapExprWrapper, validSegments, invalidSegments,
partitionsToPrune, className, false);
List<ExtendedBlocklet> prunedBlocklets = dataMapJob.execute(dataMapFormat, resolver);
// Apply expression on the blocklets.
prunedBlocklets = dataMapExprWrapper.pruneBlocklets(prunedBlocklets);
return prunedBlocklets;
}

private static SegmentStatusManager.ValidAndInvalidSegmentsInfo getValidAndInvalidSegments(
CarbonTable carbonTable) throws IOException {
SegmentStatusManager ssm = new SegmentStatusManager(carbonTable.getAbsoluteTableIdentifier());
return ssm.getValidAndInvalidSegments();
}

}
Expand Up @@ -14,24 +14,22 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.carbondata.hadoop.api;
package org.apache.carbondata.core.datamap;

import java.io.IOException;
import java.io.Serializable;
import java.util.ArrayList;
import java.util.Collections;
import java.util.Iterator;
import java.util.List;

import org.apache.carbondata.core.datamap.DataMapStoreManager;
import org.apache.carbondata.core.datamap.Segment;
import org.apache.carbondata.core.datamap.TableDataMap;
import org.apache.carbondata.core.datamap.dev.expr.DataMapDistributableWrapper;
import org.apache.carbondata.core.datamap.dev.expr.DataMapExprWrapper;
import org.apache.carbondata.core.indexstore.ExtendedBlocklet;
import org.apache.carbondata.core.indexstore.PartitionSpec;
import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
import org.apache.carbondata.core.scan.filter.resolver.FilterResolverIntf;
import org.apache.carbondata.hadoop.util.ObjectSerializationUtil;
import org.apache.carbondata.core.util.ObjectSerializationUtil;

import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.mapreduce.InputSplit;
Expand All @@ -54,18 +52,27 @@ public class DistributableDataMapFormat extends FileInputFormat<Void, ExtendedBl

private List<Segment> validSegments;

private String className;
private List<Segment> invalidSegments;

private List<PartitionSpec> partitions;

DistributableDataMapFormat(CarbonTable table,
DataMapExprWrapper dataMapExprWrapper, List<Segment> validSegments,
List<PartitionSpec> partitions, String className) {
private DataMapDistributableWrapper distributable;

private boolean isJobToClearDataMaps = false;

DistributableDataMapFormat(CarbonTable table, DataMapExprWrapper dataMapExprWrapper,
List<Segment> validSegments, List<Segment> invalidSegments, List<PartitionSpec> partitions,
boolean isJobToClearDataMaps) {
this.table = table;
this.dataMapExprWrapper = dataMapExprWrapper;
this.validSegments = validSegments;
this.className = className;
this.invalidSegments = invalidSegments;
this.partitions = partitions;
this.isJobToClearDataMaps = isJobToClearDataMaps;
}

public boolean isJobToClearDataMaps() {
return isJobToClearDataMaps;
}

public static void setFilterExp(Configuration configuration, FilterResolverIntf filterExp)
Expand Down Expand Up @@ -103,10 +110,21 @@ public RecordReader<Void, ExtendedBlocklet> createRecordReader(InputSplit inputS

@Override public void initialize(InputSplit inputSplit, TaskAttemptContext taskAttemptContext)
throws IOException, InterruptedException {
DataMapDistributableWrapper distributable = (DataMapDistributableWrapper) inputSplit;
TableDataMap dataMap = DataMapStoreManager.getInstance()
distributable = (DataMapDistributableWrapper) inputSplit;
// clear the segmentMap and from cache in executor when there are invalid segments
if (invalidSegments.size() > 0) {
DataMapStoreManager.getInstance().clearInvalidSegments(table, invalidSegments);
}
TableDataMap tableDataMap = DataMapStoreManager.getInstance()
.getDataMap(table, distributable.getDistributable().getDataMapSchema());
List<ExtendedBlocklet> blocklets = dataMap.prune(distributable.getDistributable(),
if (isJobToClearDataMaps) {
// if job is to clear datamaps just clear datamaps from cache and return
DataMapStoreManager.getInstance()
.clearDataMaps(table.getCarbonTableIdentifier().getTableUniqueName());
blockletIterator = Collections.emptyIterator();
return;
}
List<ExtendedBlocklet> blocklets = tableDataMap.prune(distributable.getDistributable(),
dataMapExprWrapper.getFilterResolverIntf(distributable.getUniqueId()), partitions);
for (ExtendedBlocklet blocklet : blocklets) {
blocklet.setDataMapUniqueId(distributable.getUniqueId());
Expand Down

0 comments on commit 2018048

Please sign in to comment.