Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

OAK-11114 - Filter downloaded Mongo documents by path suffix #1716

Merged
merged 18 commits into from
Sep 16, 2024
Merged
Show file tree
Hide file tree
Changes from 12 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
Original file line number Diff line number Diff line change
Expand Up @@ -18,9 +18,14 @@
*/
package org.apache.jackrabbit.oak.index.indexer.document.flatfile.pipelined;

import org.apache.commons.lang3.StringUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import java.util.Arrays;
import java.util.List;
import java.util.stream.Collectors;

public class ConfigHelper {
private static final Logger LOG = LoggerFactory.getLogger(ConfigHelper.class);

Expand Down Expand Up @@ -48,4 +53,18 @@ public static boolean getSystemPropertyAsBoolean(String name, boolean defaultVal
LOG.info("Config {}={}", name, value);
return value;
}

/**
* white space at the start/end of the string or at the start/end of the parts delimited by separator are trimmed
thomasmueller marked this conversation as resolved.
Show resolved Hide resolved
*/
public static List<String> getSystemPropertyAsStringList(String name, String defaultValue, char separator) {
String result = System.getProperty(name, defaultValue);
List<String> parts = splitString(result, separator);
LOG.info("Config {}={}", name, parts);
return parts;
}

private static List<String> splitString(String str, char separator) {
nfsantos marked this conversation as resolved.
Show resolved Hide resolved
return str.isBlank() ? List.of() : Arrays.stream(StringUtils.split(str, separator)).map(String::trim).collect(Collectors.toList());
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,8 @@
*/
package org.apache.jackrabbit.oak.index.indexer.document.flatfile.pipelined;

import org.apache.commons.lang3.mutable.MutableInt;
import org.apache.commons.lang3.mutable.MutableLong;
import org.apache.jackrabbit.oak.commons.IOUtils;
import org.apache.jackrabbit.oak.plugins.document.Collection;
import org.apache.jackrabbit.oak.plugins.document.Document;
import org.apache.jackrabbit.oak.plugins.document.NodeDocument;
Expand All @@ -35,9 +36,15 @@
import org.bson.codecs.DecoderContext;
import org.bson.codecs.EncoderContext;
import org.bson.codecs.configuration.CodecRegistry;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import java.util.List;
import java.util.SortedMap;
import java.util.TreeMap;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.atomic.AtomicLong;
import java.util.stream.Collectors;

/**
* Custom codec for MongoDB to transform a stream of BSON tokens into a NodeDocument. This custom codec provides two
Expand All @@ -49,44 +56,109 @@
* <li>Allows estimating the size of the document while reading it, which will have a negligible overhead (as compared
* with doing an additional traverse of the object structure to compute the size).</li>
* </ul>
*
* <p>
* This class must be thread-safe, Mongo uses a single coded implementation across multiple threads.
*
*/
public class NodeDocumentCodec implements Codec<NodeDocument> {
private final static Logger LOG = LoggerFactory.getLogger(NodeDocumentCodec.class);

public static final String OAK_INDEXER_PIPELINED_NODE_DOCUMENT_FILTER_FILTERED_PATH = "oak.indexer.pipelined.nodeDocument.filter.filteredPath";
public static final String OAK_INDEXER_PIPELINED_NODE_DOCUMENT_FILTER_SUFFIXES_TO_SKIP = "oak.indexer.pipelined.nodeDocument.filter.suffixesToSkip";
private final String filteredPath = ConfigHelper.getSystemPropertyAsString(OAK_INDEXER_PIPELINED_NODE_DOCUMENT_FILTER_FILTERED_PATH, "");
private final List<String> suffixesToSkip = ConfigHelper.getSystemPropertyAsStringList(OAK_INDEXER_PIPELINED_NODE_DOCUMENT_FILTER_SUFFIXES_TO_SKIP, "",';');

// The estimated size is stored in the NodeDocument itself
public final static String SIZE_FIELD = "_ESTIMATED_SIZE_";

private static class NodeDocumentDecoderContext {
long docsDecoded = 0;
long dataDownloaded = 0;
int estimatedSizeOfCurrentObject = 0;
}

private final NodeDocument emptyNodeDocument;

private final MongoDocumentStore store;
private final Collection<NodeDocument> collection;
private final BsonTypeCodecMap bsonTypeCodecMap;
private final DecoderContext decoderContext = DecoderContext.builder().build();

private final Codec<String> stringCoded;
private final Codec<Long> longCoded;
private final Codec<Boolean> booleanCoded;

private final NodeDocumentFilter fieldFilter = new NodeDocumentFilter(filteredPath, suffixesToSkip);

// Statistics
private final AtomicLong totalDocsDecoded = new AtomicLong(0);
private final AtomicLong totalDataDownloaded = new AtomicLong(0);
private final ThreadLocal<NodeDocumentDecoderContext> perThreadContext = ThreadLocal.withInitial(NodeDocumentDecoderContext::new);
thomasmueller marked this conversation as resolved.
Show resolved Hide resolved

public NodeDocumentCodec(MongoDocumentStore store, Collection<NodeDocument> collection, CodecRegistry defaultRegistry) {
this.store = store;
this.collection = collection;
this.bsonTypeCodecMap = new BsonTypeCodecMap(new BsonTypeClassMap(), defaultRegistry);
this.emptyNodeDocument = collection.newDocument(store);
// Retrieve references to the most commonly used codecs, to avoid the map lookup in the common case
this.stringCoded = (Codec<String>) bsonTypeCodecMap.get(BsonType.STRING);
this.longCoded = (Codec<Long>) bsonTypeCodecMap.get(BsonType.INT64);
this.booleanCoded = (Codec<Boolean>) bsonTypeCodecMap.get(BsonType.BOOLEAN);
}

/**
* Skipping over values in the BSON file is faster than reading them. Skipping is done by advancing a pointer in
* an internal buffer, while reading requires converting them to a Java data type (typically String).
*/
private void skipUntilEndOfDocument(BsonReader reader) {
nfsantos marked this conversation as resolved.
Show resolved Hide resolved
while (true) {
BsonType bsonType = reader.readBsonType();
if (bsonType == BsonType.END_OF_DOCUMENT) {
break;
}
reader.skipName();
reader.skipValue();
}
nfsantos marked this conversation as resolved.
Show resolved Hide resolved
reader.readEndDocument();
}

@Override
public NodeDocument decode(BsonReader reader, DecoderContext decoderContext) {
NodeDocument nodeDocument = collection.newDocument(store);
MutableInt estimatedSizeOfCurrentObject = new MutableInt(0);
NodeDocumentDecoderContext threadLocalContext = perThreadContext.get();
threadLocalContext.estimatedSizeOfCurrentObject = 0;
reader.readStartDocument();
while (reader.readBsonType() != BsonType.END_OF_DOCUMENT) {
String fieldName = reader.readName();
Object value = readValue(reader, fieldName, estimatedSizeOfCurrentObject);
Object value = readValue(reader, fieldName, threadLocalContext);
// Once we read the _id or the _path, apply the filter
if (fieldName.equals(NodeDocument.ID) || fieldName.equals(NodeDocument.PATH)) {
if (fieldFilter.shouldSkip(fieldName, (String) value)) {
thomasmueller marked this conversation as resolved.
Show resolved Hide resolved
skipUntilEndOfDocument(reader);
// The Mongo driver requires us to return a document. To indicate that the document should be skipped,
// we return an empty document. The logic reading from the Mongo cursor can then check if the _id of
// the document is null, which indicates that the document should be skipped.
return emptyNodeDocument;
}
}
nodeDocument.put(fieldName, value);
}
reader.readEndDocument();
nodeDocument.put(SIZE_FIELD, estimatedSizeOfCurrentObject.toInteger());
threadLocalContext.docsDecoded++;
threadLocalContext.dataDownloaded += threadLocalContext.estimatedSizeOfCurrentObject;
long docsDecodedLocal = totalDocsDecoded.incrementAndGet();
long dataDownloadedLocal = totalDataDownloaded.addAndGet(threadLocalContext.estimatedSizeOfCurrentObject);
if (docsDecodedLocal % 200_000 == 0) {
ConcurrentHashMap<String, MutableLong> filteredSuffixes = fieldFilter.getFilteredSuffixesCounts();
long totalDocumentsFiltered = filteredSuffixes.values().stream().mapToLong(MutableLong::longValue).sum();
String filteredRenditionsString = filteredSuffixes.entrySet().stream()
.sorted((e1, e2) -> e2.getValue().compareTo(e1.getValue()))
.map(e -> e.getKey() + "=" + e.getValue())
.collect(Collectors.joining(", ", "{", "}"));
LOG.info("docsDecodedThread: {}, dataDownloadedThread: {}, docsDecodedTotal: {}, dataDownloadedTotal: {}, docsSkippedTotal {}, filteredRenditionsTotal: {}",
threadLocalContext.docsDecoded, IOUtils.humanReadableByteCountBin(threadLocalContext.dataDownloaded),
totalDocsDecoded, IOUtils.humanReadableByteCountBin(dataDownloadedLocal),
totalDocumentsFiltered, filteredRenditionsString);
}
nodeDocument.put(SIZE_FIELD, threadLocalContext.estimatedSizeOfCurrentObject);
return nodeDocument;
}

Expand All @@ -100,7 +172,7 @@ public Class<NodeDocument> getEncoderClass() {
return NodeDocument.class;
}

private Object readValue(BsonReader reader, String fieldName, MutableInt estimatedSizeOfCurrentObject) {
private Object readValue(BsonReader reader, String fieldName, NodeDocumentDecoderContext threadContext) {
BsonType bsonType = reader.getCurrentBsonType();
Object value;
int valSize;
Expand All @@ -115,7 +187,7 @@ private Object readValue(BsonReader reader, String fieldName, MutableInt estimat
valSize = 16;
break;
case DOCUMENT:
value = readDocument(reader, estimatedSizeOfCurrentObject);
value = readDocument(reader, threadContext);
valSize = 0; // the size is updated by the recursive calls inside readDocument
break;
case BOOLEAN:
Expand All @@ -141,16 +213,16 @@ private Object readValue(BsonReader reader, String fieldName, MutableInt estimat
}
break;
}
estimatedSizeOfCurrentObject.add(16 + fieldName.length() + valSize);
threadContext.estimatedSizeOfCurrentObject += 16 + fieldName.length() + valSize;
return value;
}

private SortedMap<Revision, Object> readDocument(BsonReader reader, MutableInt estimatedSizeOfCurrentObject) {
private SortedMap<Revision, Object> readDocument(BsonReader reader, NodeDocumentDecoderContext threadContext) {
TreeMap<Revision, Object> map = new TreeMap<>(StableRevisionComparator.REVERSE);
reader.readStartDocument();
while (reader.readBsonType() != BsonType.END_OF_DOCUMENT) {
String fieldName = reader.readName();
Object value = readValue(reader, fieldName, estimatedSizeOfCurrentObject);
Object value = readValue(reader, fieldName, threadContext);
map.put(Revision.fromString(fieldName), value);
}
reader.readEndDocument();
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,115 @@
/*
* 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.jackrabbit.oak.index.indexer.document.flatfile.pipelined;

import org.apache.commons.lang3.mutable.MutableLong;
import org.apache.jackrabbit.oak.plugins.document.NodeDocument;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import java.util.List;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.atomic.AtomicLong;

/**
* Implements a filter to decide if a given Mongo document should be processed or ignored based on its path. The filter has
* two configuration parameters:
*
* <ul>
* <li> filteredPath - The path where the filter is applied. Only the documents inside this path will be considered for filtering.
* Documents in other paths will all be accepted.
* <li> suffixesToSkip - A list of suffixes to filter. That is, any document whose path ends in one of these suffixes will
* be filtered.
* </ul>
* <p>
* The intent of this filter is to be applied as close as possible to the download/decoding of the documents from Mongo,
* in order to filter unnecessary documents early and avoid spending resources processing them.
*/
public class NodeDocumentFilter {
private static final Logger LOG = LoggerFactory.getLogger(NodeDocumentFilter.class);

private final String filteredPath;
private final List<String> suffixesToSkip;

private final boolean filteringDisabled;

// Statistics
private final AtomicLong skippedFields = new AtomicLong(0);
private final AtomicLong longPathSkipped = new AtomicLong(0);
private final ConcurrentHashMap<String, MutableLong> filteredSuffixesCounts = new ConcurrentHashMap<>();

public NodeDocumentFilter(String filteredPath, List<String> suffixesToSkip) {
this.filteredPath = filteredPath;
this.suffixesToSkip = suffixesToSkip;
this.filteringDisabled = filteredPath.isBlank() || suffixesToSkip.isEmpty();
if (filteringDisabled) {
LOG.info("Node document filtering disabled.");
}
}

/**
* @param fieldName Name of the Mongo document field. Expected to be either _id or _path
* @param idOrPathValue The value of the field
* @return true if the document should be skipped, false otherwise
*/
public boolean shouldSkip(String fieldName, String idOrPathValue) {
if (filteringDisabled) {
return false;
}
// Check if the NodeDocument should be considered for filtering, that is, if it starts with includePath.
// If the value is for an _id, then we must find the start of the path section, that is, the position of the first
// slash (3:/foo/bar/baz). If the value given is for a path, then it already contains only the path. In any case,
// we look up for the first occurrence of /
int idxOfFirstForwardSlash = idOrPathValue.indexOf('/');
if (idxOfFirstForwardSlash < 0) {
LOG.warn("Invalid field. {} = {}", fieldName, idOrPathValue);
return false;
}
if (idOrPathValue.startsWith(filteredPath, idxOfFirstForwardSlash)) {
// Match the include path. Check if it ends with any of the suffixes to skip.
for (String suffix : suffixesToSkip) {
if (idOrPathValue.endsWith(suffix)) {
// This node document should be skipped.
filteredSuffixesCounts.computeIfAbsent(suffix, k -> new MutableLong(0)).increment();
long skippedSoFar = skippedFields.incrementAndGet();
if (fieldName.equals(NodeDocument.PATH)) {
longPathSkipped.incrementAndGet();
}
if (skippedSoFar % 50_000 == 0) {
LOG.info("skippedSoFar: {}. Long path: {}, Doc: {}={}", skippedSoFar, longPathSkipped.get(), fieldName, idOrPathValue);
}
return true;
}
}
}
return false;
}

public long getSkippedFields() {
return skippedFields.get();
}

public long getLongPathSkipped() {
return longPathSkipped.get();
}

public ConcurrentHashMap<String, MutableLong> getFilteredSuffixesCounts() {
return filteredSuffixesCounts;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -786,7 +786,11 @@ void download(FindIterable<NodeDocument> mongoIterable) throws InterruptedExcept
try {
while (cursor.hasNext()) {
NodeDocument next = cursor.next();
// If the id is not set, then the document was filtered by NodeDocumentFilter and should be ignored
String id = next.getId();
if (id == null) {
continue;
}
// All the Mongo queries in this class have a requirement on the _modified field, so the
// documents downloaded will all have the field defined.
this.nextLastModified = next.getModified();
Expand All @@ -796,7 +800,7 @@ void download(FindIterable<NodeDocument> mongoIterable) throws InterruptedExcept
this.lastIdDownloaded = id;
this.documentsDownloadedTotal++;
downloadStatics.incrementDocumentsDownloadedTotal();
if (this.documentsDownloadedTotal % 20_000 == 0) {
if (this.documentsDownloadedTotal % 50_000 == 0) {
Copy link
Contributor

Choose a reason for hiding this comment

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

Can you extract 50_000 to constant?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

We are inlining similar constants in other places. Maybe it is not a good practise, but it's ok for the time being, I don't want to expose this as a constant.

Additionally, the whole approach to logging progress should be reconsidered. I no longer think that logging based on the number of items processed is the best approach, as download speeds vary a lot from run to run, so we risk either not logging often enough or logging way too frequently. A better approach is to log periodically, so at a certain point in the future I would like to revise the whole approach to logging progress in the indexing job.

Copy link
Member

Choose a reason for hiding this comment

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

No, I think this is not needed. Actually I'm against making it a constant: that would just complicate the code without giving any meaningful value, in my view. (I understand others might have a different opinion, that's fine.)

reportProgress(id);
}
TRAVERSAL_LOG.trace(id);
Expand Down
Loading
Loading