Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
Add cardinality estimation for predicting disk consumption of active …
…log entries for efficient log compaction.
- Loading branch information
Showing
10 changed files
with
289 additions
and
31 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
105 changes: 105 additions & 0 deletions
105
raft/src/main/java/net/kuujo/copycat/protocol/raft/storage/KeySet.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Original file line | Diff line number | Diff line change |
---|---|---|---|
@@ -0,0 +1,105 @@ | |||
/* | |||
* Copyright 2015 the original author or authors. | |||
* | |||
* Licensed 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 net.kuujo.copycat.protocol.raft.storage; | |||
|
|||
import com.clearspring.analytics.stream.cardinality.CardinalityMergeException; | |||
import com.clearspring.analytics.stream.cardinality.HyperLogLogPlus; | |||
import com.clearspring.analytics.stream.cardinality.ICardinality; | |||
import net.kuujo.copycat.io.Buffer; | |||
import net.kuujo.copycat.io.util.HashFunction; | |||
import net.kuujo.copycat.io.util.Murmur3HashFunction; | |||
|
|||
import java.io.IOException; | |||
|
|||
/** | |||
* Segment key set. | |||
* <p> | |||
* The key set tracks the total number of unique keys within a given segment using a memory efficient estimator algorithm. | |||
* Specifically, this class uses AddThis's HyperLogLog++ implementation. | |||
* | |||
* @author <a href="http://github.com/kuujo">Jordan Halterman</a> | |||
*/ | |||
public class KeySet { | |||
private final HashFunction hash = new Murmur3HashFunction(); | |||
private final ICardinality cardinality; | |||
|
|||
KeySet() { | |||
cardinality = new HyperLogLogPlus(10, 14); | |||
} | |||
|
|||
public KeySet(byte[] bytes) { | |||
try { | |||
cardinality = HyperLogLogPlus.Builder.build(bytes); | |||
} catch (IOException e) { | |||
throw new StorageException("failed to deserialize cardinality estimator"); | |||
} | |||
} | |||
|
|||
private KeySet(ICardinality cardinality) { | |||
this.cardinality = cardinality; | |||
} | |||
|
|||
/** | |||
* Returns the cardinality of the set. | |||
* | |||
* @return The cardinality of the set. | |||
*/ | |||
public long cardinality() { | |||
return cardinality.cardinality(); | |||
} | |||
|
|||
/** | |||
* Adds a key to the set. | |||
* | |||
* @param key The key to add. | |||
*/ | |||
public void add(Buffer key) { | |||
cardinality.offerHashed(hash.hash64(key)); | |||
} | |||
|
|||
/** | |||
* Merges two key sets together. | |||
* | |||
* @param keySet The key set to merge in to this key set. | |||
* @return The merged key set. | |||
*/ | |||
public KeySet merge(KeySet keySet) { | |||
try { | |||
return new KeySet(cardinality.merge(keySet.cardinality)); | |||
} catch (CardinalityMergeException e) { | |||
throw new StorageException("failed to merge cardinality estimators", e); | |||
} | |||
} | |||
|
|||
/** | |||
* Serializes the key set. | |||
* | |||
* @return The serialized key set. | |||
*/ | |||
public byte[] getBytes() { | |||
try { | |||
return cardinality.getBytes(); | |||
} catch (IOException e) { | |||
throw new StorageException("failed to serialize cardinality estimator", e); | |||
} | |||
} | |||
|
|||
@Override | |||
public String toString() { | |||
return String.format("KeySet[cardinality=%d]", cardinality.cardinality()); | |||
} | |||
|
|||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
72 changes: 72 additions & 0 deletions
72
...n/java/net/kuujo/copycat/protocol/raft/storage/compact/KeyCountingCompactionStrategy.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Original file line | Diff line number | Diff line change |
---|---|---|---|
@@ -0,0 +1,72 @@ | |||
/* | |||
* Copyright 2015 the original author or authors. | |||
* | |||
* Licensed 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 net.kuujo.copycat.protocol.raft.storage.compact; | |||
|
|||
import net.kuujo.copycat.protocol.raft.storage.KeySet; | |||
import net.kuujo.copycat.protocol.raft.storage.Segment; | |||
import org.slf4j.Logger; | |||
import org.slf4j.LoggerFactory; | |||
|
|||
import java.util.ArrayList; | |||
import java.util.List; | |||
import java.util.stream.Collectors; | |||
|
|||
/** | |||
* Cardinality based compaction strategy. | |||
* | |||
* @author <a href="http://github.com/kuujo">Jordan Halterman</a> | |||
*/ | |||
public class KeyCountingCompactionStrategy extends LeveledCompactionStrategy { | |||
private static final Logger LOGGER = LoggerFactory.getLogger(KeyCountingCompactionStrategy.class); | |||
|
|||
@Override | |||
protected Logger logger() { | |||
return LOGGER; | |||
} | |||
|
|||
@Override | |||
protected List<List<Segment>> selectSegments(List<Segment> segments) { | |||
return super.selectSegments(segments).stream() | |||
.map(l -> calculateCompactSegments(l, config.getEntriesPerSegment())) | |||
.collect(Collectors.toList()); | |||
} | |||
|
|||
/** | |||
* Calculates the set of segments to compact for the given level segments. | |||
*/ | |||
private List<Segment> calculateCompactSegments(List<Segment> segments, int entriesPerSegment) { | |||
// Determine the set of segments that can be compacted from the level by estimating the cardinality of the oldest | |||
// set of segments up to the maximum number of entries allowed in a segment. | |||
List<Segment> compactSegments = new ArrayList<>(); | |||
KeySet keys = null; | |||
for (Segment segment : segments) { | |||
if (keys == null) { | |||
keys = segment.keys(); | |||
compactSegments.add(segment); | |||
} else { | |||
keys = keys.merge(segment.keys()); | |||
|
|||
if (keys.cardinality() > entriesPerSegment) { | |||
return compactSegments; | |||
} else { | |||
compactSegments.add(segment); | |||
} | |||
} | |||
} | |||
return compactSegments; | |||
} | |||
|
|||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Oops, something went wrong.