Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
12 changes: 6 additions & 6 deletions docs/content/development/extensions-core/bloom-filter.md
Original file line number Diff line number Diff line change
Expand Up @@ -23,7 +23,7 @@ Internally, this implementation of bloom filter uses Murmur3 fast non-cryptograp
{
"type" : "bloom",
"dimension" : <dimension_name>,
"bloomKFilter" : <serialized_bytes_for_BloomKFilter>,
"bloomFilter" : <serialized_bytes_for_BloomFilter>,
"extractionFn" : <extraction_fn>
}
```
Expand All @@ -32,14 +32,14 @@ Internally, this implementation of bloom filter uses Murmur3 fast non-cryptograp
|-------------------------|------------------------------|----------------------------------|
|`type` |Filter Type. Should always be `bloom`|yes|
|`dimension` |The dimension to filter over. | yes |
|`bloomKFilter` |Base64 encoded Binary representation of `org.apache.hive.common.util.BloomKFilter`| yes |
|`bloomFilter` |Base64 encoded Binary representation of `org.apache.hive.common.util.BloomFilter`| yes |
|`extractionFn`|[Extraction function](./../dimensionspecs.html#extraction-functions) to apply to the dimension values |no|


### Serialized Format for BloomKFilter
Serialized BloomKFilter format:
### Serialized Format for BloomFilter
Serialized BloomFilter format:
- 1 byte for the number of hash functions.
- 1 big endian int(That is how OutputStream works) for the number of longs in the bitset
- big endian longs in the BloomKFilter bitset
- big endian longs in the BloomFilter bitset

Note: `org.apache.hive.common.util.BloomKFilter` provides a serialize method which can be used to serialize bloom filters to outputStream.
Note: `org.apache.hive.common.util.BloomFilter` provides a serialize method which can be used to serialize bloom filters to outputStream.
Original file line number Diff line number Diff line change
Expand Up @@ -27,8 +27,8 @@
import com.fasterxml.jackson.databind.module.SimpleModule;
import com.fasterxml.jackson.databind.ser.std.StdSerializer;
import org.apache.druid.query.filter.BloomDimFilter;
import org.apache.druid.query.filter.BloomKFilterHolder;
import org.apache.hive.common.util.BloomKFilter;
import org.apache.druid.query.filter.BloomFilterHolder;
import org.apache.hive.common.util.BloomFilter;

import java.io.ByteArrayInputStream;
import java.io.ByteArrayOutputStream;
Expand All @@ -41,65 +41,66 @@ public class BloomFilterSerializersModule extends SimpleModule
public BloomFilterSerializersModule()
{
registerSubtypes(new NamedType(BloomDimFilter.class, BLOOM_FILTER_TYPE_NAME));
addSerializer(BloomKFilter.class, new BloomKFilterSerializer());
addDeserializer(BloomKFilter.class, new BloomKFilterDeserializer());
addDeserializer(BloomKFilterHolder.class, new BloomKFilterHolderDeserializer());
addSerializer(BloomFilter.class, new BloomFilterSerializer());
addDeserializer(BloomFilter.class, new BloomFilterDeserializer());
addDeserializer(BloomFilterHolder.class, new BloomFilterHolderDeserializer());
}

private static class BloomKFilterSerializer extends StdSerializer<BloomKFilter>
private static class BloomFilterSerializer extends StdSerializer<BloomFilter>
{
BloomKFilterSerializer()
BloomFilterSerializer()
{
super(BloomKFilter.class);
super(BloomFilter.class);
}

@Override
public void serialize(BloomKFilter bloomKFilter, JsonGenerator jsonGenerator, SerializerProvider serializerProvider)
public void serialize(BloomFilter bloomFilter, JsonGenerator jsonGenerator, SerializerProvider serializerProvider)
throws IOException
{
jsonGenerator.writeBinary(bloomKFilterToBytes(bloomKFilter));
jsonGenerator.writeBinary(bloomFilterToBytes(bloomFilter));
}
}

private static class BloomKFilterDeserializer extends StdDeserializer<BloomKFilter>
private static class BloomFilterDeserializer extends StdDeserializer<BloomFilter>
{
BloomKFilterDeserializer()
BloomFilterDeserializer()
{
super(BloomKFilter.class);
super(BloomFilter.class);
}

@Override
public BloomKFilter deserialize(JsonParser jsonParser, DeserializationContext deserializationContext)
public BloomFilter deserialize(JsonParser jsonParser, DeserializationContext deserializationContext)
throws IOException
{
return bloomKFilterFromBytes(jsonParser.getBinaryValue());
return bloomFilterFromBytes(jsonParser.getBinaryValue());
}
}

private static class BloomKFilterHolderDeserializer extends StdDeserializer<BloomKFilterHolder>
private static class BloomFilterHolderDeserializer extends StdDeserializer<BloomFilterHolder>
{
BloomKFilterHolderDeserializer()
BloomFilterHolderDeserializer()
{
super(BloomKFilterHolder.class);
super(BloomFilterHolder.class);
}

@Override
public BloomKFilterHolder deserialize(JsonParser jsonParser, DeserializationContext deserializationContext)
public BloomFilterHolder deserialize(JsonParser jsonParser, DeserializationContext deserializationContext)
throws IOException
{
return BloomKFilterHolder.fromBytes(jsonParser.getBinaryValue());
byte[] bytes = jsonParser.getBinaryValue();
return BloomFilterHolder.fromBytes(bytes);
}
}

public static byte[] bloomKFilterToBytes(BloomKFilter bloomKFilter) throws IOException
public static byte[] bloomFilterToBytes(BloomFilter bloomFilter) throws IOException
{
ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream();
BloomKFilter.serialize(byteArrayOutputStream, bloomKFilter);
BloomFilter.serialize(byteArrayOutputStream, bloomFilter);
return byteArrayOutputStream.toByteArray();
}

public static BloomKFilter bloomKFilterFromBytes(byte[] bytes) throws IOException
public static BloomFilter bloomFilterFromBytes(byte[] bytes) throws IOException
{
return BloomKFilter.deserialize(new ByteArrayInputStream(bytes));
return BloomFilter.deserialize(new ByteArrayInputStream(bytes));
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -30,7 +30,7 @@
import org.apache.druid.query.cache.CacheKeyBuilder;
import org.apache.druid.query.extraction.ExtractionFn;
import org.apache.druid.segment.filter.DimensionPredicateFilter;
import org.apache.hive.common.util.BloomKFilter;
import org.apache.hive.common.util.BloomFilter;

import java.util.HashSet;

Expand All @@ -40,22 +40,22 @@ public class BloomDimFilter implements DimFilter
{

private final String dimension;
private final BloomKFilter bloomKFilter;
private final BloomFilter bloomFilter;
private final HashCode hash;
private final ExtractionFn extractionFn;

@JsonCreator
public BloomDimFilter(
@JsonProperty("dimension") String dimension,
@JsonProperty("bloomKFilter") BloomKFilterHolder bloomKFilterHolder,
@JsonProperty("bloomFilter") BloomFilterHolder bloomFilterHolder,
@JsonProperty("extractionFn") ExtractionFn extractionFn
)
{
Preconditions.checkArgument(dimension != null, "dimension must not be null");
Preconditions.checkNotNull(bloomKFilterHolder);
Preconditions.checkNotNull(bloomFilterHolder);
this.dimension = dimension;
this.bloomKFilter = bloomKFilterHolder.getFilter();
this.hash = bloomKFilterHolder.getFilterHash();
this.bloomFilter = bloomFilterHolder.getFilter();
this.hash = bloomFilterHolder.getFilterHash();
this.extractionFn = extractionFn;
}

Expand Down Expand Up @@ -90,9 +90,9 @@ public Predicate<String> makeStringPredicate()
{
return str -> {
if (str == null) {
return bloomKFilter.testBytes(null, 0, 0);
return bloomFilter.testBytes(null, 0, 0);
}
return bloomKFilter.testString(str);
return bloomFilter.testString(str);
};
}

Expand All @@ -104,13 +104,13 @@ public DruidLongPredicate makeLongPredicate()
@Override
public boolean applyLong(long input)
{
return bloomKFilter.testLong(input);
return bloomFilter.testLong(input);
}

@Override
public boolean applyNull()
{
return bloomKFilter.testBytes(null, 0, 0);
return bloomFilter.testBytes(null, 0, 0);
}
};
}
Expand All @@ -123,13 +123,13 @@ public DruidFloatPredicate makeFloatPredicate()
@Override
public boolean applyFloat(float input)
{
return bloomKFilter.testFloat(input);
return bloomFilter.testDouble(input);
Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I especially don't like this change, going to try and find something better to do here.

}

@Override
public boolean applyNull()
{
return bloomKFilter.testBytes(null, 0, 0);
return bloomFilter.testBytes(null, 0, 0);
}
};
}
Expand All @@ -142,13 +142,13 @@ public DruidDoublePredicate makeDoublePredicate()
@Override
public boolean applyDouble(double input)
{
return bloomKFilter.testDouble(input);
return bloomFilter.testDouble(input);
}

@Override
public boolean applyNull()
{
return bloomKFilter.testBytes(null, 0, 0);
return bloomFilter.testBytes(null, 0, 0);
}
};
}
Expand All @@ -164,9 +164,9 @@ public String getDimension()
}

@JsonProperty
public BloomKFilter getBloomKFilter()
public BloomFilter getBloomFilter()
{
return bloomKFilter;
return bloomFilter;
}

@JsonProperty
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,23 +22,23 @@
import com.google.common.hash.HashCode;
import com.google.common.hash.Hashing;
import org.apache.druid.guice.BloomFilterSerializersModule;
import org.apache.hive.common.util.BloomKFilter;
import org.apache.hive.common.util.BloomFilter;

import java.io.IOException;
import java.util.Objects;

public class BloomKFilterHolder
public class BloomFilterHolder
{
private final BloomKFilter filter;
private final BloomFilter filter;
private final HashCode hash;

public BloomKFilterHolder(BloomKFilter filter, HashCode hash)
public BloomFilterHolder(BloomFilter filter, HashCode hash)
{
this.filter = filter;
this.hash = hash;
}

BloomKFilter getFilter()
BloomFilter getFilter()
{
return filter;
}
Expand All @@ -48,17 +48,17 @@ HashCode getFilterHash()
return hash;
}

public static BloomKFilterHolder fromBloomKFilter(BloomKFilter filter) throws IOException
public static BloomFilterHolder fromBloomFilter(BloomFilter filter) throws IOException
{
byte[] bytes = BloomFilterSerializersModule.bloomKFilterToBytes(filter);
byte[] bytes = BloomFilterSerializersModule.bloomFilterToBytes(filter);

return new BloomKFilterHolder(filter, Hashing.sha512().hashBytes(bytes));
return new BloomFilterHolder(filter, Hashing.sha512().hashBytes(bytes));
}

public static BloomKFilterHolder fromBytes(byte[] bytes) throws IOException
public static BloomFilterHolder fromBytes(byte[] bytes) throws IOException
{
return new BloomKFilterHolder(
BloomFilterSerializersModule.bloomKFilterFromBytes(bytes),
return new BloomFilterHolder(
BloomFilterSerializersModule.bloomFilterFromBytes(bytes),
Hashing.sha512().hashBytes(bytes)
);
}
Expand All @@ -73,7 +73,7 @@ public boolean equals(Object o)
return false;
}

BloomKFilterHolder that = (BloomKFilterHolder) o;
BloomFilterHolder that = (BloomFilterHolder) o;
return Objects.equals(this.hash, that.hash);
}

Expand Down
Loading