Skip to content

Commit

Permalink
Merge 25ab027 into 8427771
Browse files Browse the repository at this point in the history
  • Loading branch information
kevinjmh committed Sep 28, 2018
2 parents 8427771 + 25ab027 commit 668ec60
Showing 1 changed file with 55 additions and 45 deletions.
Expand Up @@ -21,15 +21,7 @@
import java.io.UnsupportedEncodingException;
import java.text.DateFormat;
import java.text.SimpleDateFormat;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Date;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.TimeZone;
import java.util.*;
import java.util.concurrent.ConcurrentHashMap;

import org.apache.carbondata.common.annotations.InterfaceAudience;
Expand All @@ -47,7 +39,6 @@
import org.apache.carbondata.core.indexstore.PartitionSpec;
import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
import org.apache.carbondata.core.metadata.CarbonMetadata;
import org.apache.carbondata.core.metadata.datatype.DataType;
import org.apache.carbondata.core.metadata.datatype.DataTypes;
import org.apache.carbondata.core.metadata.encoder.Encoding;
import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
Expand Down Expand Up @@ -184,6 +175,7 @@ public List<Blocklet> prune(FilterResolverIntf filterExp, SegmentProperties segm
throw new RuntimeException(e);
}
for (BloomQueryModel bloomQueryModel : bloomQueryModels) {
Set<Blocklet> tempHitBlockletsResult = new HashSet<>();
LOGGER.debug("prune blocklet for query: " + bloomQueryModel);
BloomCacheKeyValue.CacheKey cacheKey = new BloomCacheKeyValue.CacheKey(
this.indexPath.toString(), bloomQueryModel.columnName);
Expand All @@ -194,17 +186,22 @@ public List<Blocklet> prune(FilterResolverIntf filterExp, SegmentProperties segm
// skip shard which has been pruned in Main datamap
continue;
}
boolean scanRequired = bloomFilter.membershipTest(new Key(bloomQueryModel.filterValue));
if (scanRequired) {
LOGGER.debug(String.format("BloomCoarseGrainDataMap: Need to scan -> blocklet#%s",
String.valueOf(bloomFilter.getBlockletNo())));
Blocklet blocklet = new Blocklet(bloomFilter.getShardName(),
String.valueOf(bloomFilter.getBlockletNo()));
hitBlocklets.add(blocklet);
} else {
LOGGER.debug(String.format("BloomCoarseGrainDataMap: Skip scan -> blocklet#%s",
String.valueOf(bloomFilter.getBlockletNo())));
for (byte[] value: bloomQueryModel.filterValues) {
boolean scanRequired = bloomFilter.membershipTest(new Key(value));
if (scanRequired) {
LOGGER.debug(String.format("BloomCoarseGrainDataMap: Need to scan -> blocklet#%s",
String.valueOf(bloomFilter.getBlockletNo())));
Blocklet blocklet = new Blocklet(bloomFilter.getShardName(),
String.valueOf(bloomFilter.getBlockletNo()));
tempHitBlockletsResult.add(blocklet);
} else {
LOGGER.debug(String.format("BloomCoarseGrainDataMap: Skip scan -> blocklet#%s",
String.valueOf(bloomFilter.getBlockletNo())));
}
}
// get intersect result between query models
// pre-condition: only And/In/EqualTo expression exists in single bloom datamap
hitBlocklets.retainAll(tempHitBlockletsResult);
}
}
return new ArrayList<>(hitBlocklets);
Expand Down Expand Up @@ -246,17 +243,17 @@ private List<BloomQueryModel> createQueryModel(Expression expression)
if (left instanceof ColumnExpression && right instanceof ListExpression) {
column = ((ColumnExpression) left).getColumnName();
if (this.name2Col.containsKey(column)) {
List<BloomQueryModel> models =
BloomQueryModel bloomQueryModel =
buildQueryModelForIn((ColumnExpression) left, (ListExpression) right);
queryModels.addAll(models);
queryModels.add(bloomQueryModel);
}
return queryModels;
} else if (left instanceof ListExpression && right instanceof ColumnExpression) {
column = ((ColumnExpression) right).getColumnName();
if (this.name2Col.containsKey(column)) {
List<BloomQueryModel> models =
BloomQueryModel bloomQueryModel =
buildQueryModelForIn((ColumnExpression) right, (ListExpression) left);
queryModels.addAll(models);
queryModels.add(bloomQueryModel);
}
return queryModels;
} else {
Expand All @@ -272,21 +269,22 @@ private List<BloomQueryModel> createQueryModel(Expression expression)
return queryModels;
}

private BloomQueryModel buildQueryModelForEqual(ColumnExpression ce,
LiteralExpression le) throws DictionaryGenerationException, UnsupportedEncodingException {
String columnName = ce.getColumnName();
DataType dataType = ce.getDataType();
/**
* Here preprocessed NULL and date/timestamp data type.
*
* Note that if the datatype is date/timestamp, the expressionValue is long type.
*/
private Object getLiteralExpValue(LiteralExpression le) {
Object expressionValue = le.getLiteralExpValue();
Object literalValue;
// note that if the datatype is date/timestamp, the expressionValue is long type.

if (null == expressionValue) {
literalValue = null;
} else if (le.getLiteralExpDataType() == DataTypes.DATE) {
DateFormat format = new SimpleDateFormat(CarbonCommonConstants.CARBON_DATE_DEFAULT_FORMAT);
// the below settings are set statically according to DateDirectDirectionaryGenerator
format.setLenient(false);
format.setTimeZone(TimeZone.getTimeZone("GMT"));

literalValue = format.format(new Date((long) expressionValue / 1000));
} else if (le.getLiteralExpDataType() == DataTypes.TIMESTAMP) {
DateFormat format =
Expand All @@ -297,28 +295,37 @@ private BloomQueryModel buildQueryModelForEqual(ColumnExpression ce,
} else {
literalValue = expressionValue;
}
return literalValue;
}


return buildQueryModelInternal(this.name2Col.get(columnName), literalValue, dataType);
private BloomQueryModel buildQueryModelForEqual(ColumnExpression ce,
LiteralExpression le) throws DictionaryGenerationException, UnsupportedEncodingException {
List<byte[]> filterValues = new ArrayList<>();
byte[] internalFilterValue = getInternalFilterValue(this.name2Col.get(ce.getColumnName()), le);
filterValues.add(internalFilterValue);
return new BloomQueryModel(ce.getColumnName(), filterValues);
}

/**
* for `in` expressions, we use `equal` to handle it.
* Note that `in` operator needs at least one match not exactly match. since while doing pruning,
* we collect all the blocklets that will match the querymodel, this will not be a problem.
*/
private List<BloomQueryModel> buildQueryModelForIn(ColumnExpression ce, ListExpression le)
private BloomQueryModel buildQueryModelForIn(ColumnExpression ce, ListExpression le)
throws DictionaryGenerationException, UnsupportedEncodingException {
List<BloomQueryModel> queryModels = new ArrayList<>();
List<byte[]> filterValues = new ArrayList<>();
for (Expression child : le.getChildren()) {
queryModels.add(buildQueryModelForEqual(ce, (LiteralExpression) child));
byte[] internalFilterValue = getInternalFilterValue(
this.name2Col.get(ce.getColumnName()), (LiteralExpression) child);
filterValues.add(internalFilterValue);
}
return queryModels;
return new BloomQueryModel(ce.getColumnName(), filterValues);
}

private BloomQueryModel buildQueryModelInternal(CarbonColumn carbonColumn,
Object filterLiteralValue, DataType filterValueDataType) throws
private byte[] getInternalFilterValue(CarbonColumn carbonColumn, LiteralExpression le) throws
DictionaryGenerationException, UnsupportedEncodingException {
// convert the filter value to string and apply convertes on it to get carbon internal value
Object filterLiteralValue = getLiteralExpValue(le);
// convert the filter value to string and apply converters on it to get carbon internal value
String strFilterValue = null;
if (null != filterLiteralValue) {
strFilterValue = String.valueOf(filterLiteralValue);
Expand Down Expand Up @@ -360,7 +367,7 @@ private BloomQueryModel buildQueryModelInternal(CarbonColumn carbonColumn,
if (internalFilterValue.length == 0) {
internalFilterValue = CarbonCommonConstants.MEMBER_DEFAULT_VAL_ARRAY;
}
return new BloomQueryModel(carbonColumn.getColName(), internalFilterValue);
return internalFilterValue;
}

@Override
Expand All @@ -374,25 +381,28 @@ public void clear() {

static class BloomQueryModel {
private String columnName;
private byte[] filterValue;
private List<byte[]> filterValues;

/**
* represent an query model will be applyied on bloom index
*
* @param columnName bloom index column
* @param filterValue key for the bloom index,
* @param filterValues key for the bloom index,
* this value is converted from user specified filter value in query
*/
private BloomQueryModel(String columnName, byte[] filterValue) {
private BloomQueryModel(String columnName, List<byte[]> filterValues) {
this.columnName = columnName;
this.filterValue = filterValue;
this.filterValues = filterValues;
}

@Override
public String toString() {
final StringBuilder sb = new StringBuilder("BloomQueryModel{");
sb.append("columnName='").append(columnName).append('\'');
sb.append(", filterValue=").append(Arrays.toString(filterValue));
sb.append(", filterValues=");
for (byte[] value: filterValues) {
sb.append(Arrays.toString(value));
}
sb.append('}');
return sb.toString();
}
Expand Down

0 comments on commit 668ec60

Please sign in to comment.