Browse Source
Make PayloadGenerator an interface which can have multiple implementations: constant, uniform random, sequential. Allow different payload generators to be used for keys and values. This change fixes RoundTripWorkload. Previously RoundTripWorkload was unable to get the sequence number of the keys that it produced.pull/4729/head
Colin Patrick McCabe
7 years ago
committed by
Jason Gustafson
11 changed files with 422 additions and 233 deletions
@ -0,0 +1,54 @@
@@ -0,0 +1,54 @@
|
||||
/* |
||||
* 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.kafka.trogdor.workload; |
||||
|
||||
import com.fasterxml.jackson.annotation.JsonCreator; |
||||
import com.fasterxml.jackson.annotation.JsonProperty; |
||||
|
||||
/** |
||||
* A PayloadGenerator which always generates a constant payload. |
||||
*/ |
||||
public class ConstantPayloadGenerator implements PayloadGenerator { |
||||
private final int size; |
||||
private final byte[] value; |
||||
|
||||
@JsonCreator |
||||
public ConstantPayloadGenerator(@JsonProperty("size") int size, |
||||
@JsonProperty("value") byte[] value) { |
||||
this.size = size; |
||||
this.value = (value == null || value.length == 0) ? new byte[size] : value; |
||||
} |
||||
|
||||
@JsonProperty |
||||
public int size() { |
||||
return size; |
||||
} |
||||
|
||||
@JsonProperty |
||||
public byte[] value() { |
||||
return value; |
||||
} |
||||
|
||||
@Override |
||||
public byte[] generate(long position) { |
||||
byte[] next = new byte[size]; |
||||
for (int i = 0; i < next.length; i += value.length) { |
||||
System.arraycopy(value, 0, next, i, Math.min(next.length - i, value.length)); |
||||
} |
||||
return next; |
||||
} |
||||
} |
@ -0,0 +1,55 @@
@@ -0,0 +1,55 @@
|
||||
/* |
||||
* 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.kafka.trogdor.workload; |
||||
|
||||
import java.util.Iterator; |
||||
|
||||
/** |
||||
* An iterator which wraps a PayloadGenerator. |
||||
*/ |
||||
public final class PayloadIterator implements Iterator<byte[]> { |
||||
private final PayloadGenerator generator; |
||||
private long position = 0; |
||||
|
||||
public PayloadIterator(PayloadGenerator generator) { |
||||
this.generator = generator; |
||||
} |
||||
|
||||
@Override |
||||
public boolean hasNext() { |
||||
return true; |
||||
} |
||||
|
||||
@Override |
||||
public synchronized byte[] next() { |
||||
return generator.generate(position++); |
||||
} |
||||
|
||||
@Override |
||||
public void remove() { |
||||
throw new UnsupportedOperationException(); |
||||
} |
||||
|
||||
public synchronized void seek(long position) { |
||||
this.position = position; |
||||
} |
||||
|
||||
public synchronized long position() { |
||||
return this.position; |
||||
} |
||||
} |
@ -0,0 +1,65 @@
@@ -0,0 +1,65 @@
|
||||
/* |
||||
* 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.kafka.trogdor.workload; |
||||
|
||||
import com.fasterxml.jackson.annotation.JsonCreator; |
||||
import com.fasterxml.jackson.annotation.JsonProperty; |
||||
|
||||
import java.nio.ByteBuffer; |
||||
import java.nio.ByteOrder; |
||||
|
||||
/** |
||||
* A PayloadGenerator which generates a sequentially increasing payload. |
||||
* |
||||
* The generated number will wrap around to 0 after the maximum value is reached. |
||||
* Payloads bigger than 8 bytes will always just be padded with zeros after byte 8. |
||||
*/ |
||||
public class SequentialPayloadGenerator implements PayloadGenerator { |
||||
private final int size; |
||||
private final long startOffset; |
||||
private final ByteBuffer buf; |
||||
|
||||
@JsonCreator |
||||
public SequentialPayloadGenerator(@JsonProperty("size") int size, |
||||
@JsonProperty("offset") long startOffset) { |
||||
this.size = size; |
||||
this.startOffset = startOffset; |
||||
this.buf = ByteBuffer.allocate(8); |
||||
// Little-endian byte order allows us to support arbitrary lengths more easily,
|
||||
// since the first byte is always the lowest-order byte.
|
||||
this.buf.order(ByteOrder.LITTLE_ENDIAN); |
||||
} |
||||
|
||||
@JsonProperty |
||||
public int size() { |
||||
return size; |
||||
} |
||||
|
||||
@JsonProperty |
||||
public long startOffset() { |
||||
return startOffset; |
||||
} |
||||
|
||||
@Override |
||||
public synchronized byte[] generate(long position) { |
||||
buf.clear(); |
||||
buf.putLong(position + startOffset); |
||||
byte[] result = new byte[size]; |
||||
System.arraycopy(buf.array(), 0, result, 0, Math.min(buf.array().length, result.length)); |
||||
return result; |
||||
} |
||||
} |
@ -0,0 +1,89 @@
@@ -0,0 +1,89 @@
|
||||
/* |
||||
* 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.kafka.trogdor.workload; |
||||
|
||||
import com.fasterxml.jackson.annotation.JsonCreator; |
||||
import com.fasterxml.jackson.annotation.JsonProperty; |
||||
|
||||
import java.util.Random; |
||||
|
||||
/** |
||||
* A PayloadGenerator which generates a uniform random payload. |
||||
* |
||||
* This generator generates pseudo-random payloads that can be reproduced from run to run. |
||||
* The guarantees are the same as those of java.util.Random. |
||||
* |
||||
* This payload generator also has the option to append padding bytes at the end of the payload. |
||||
* The padding bytes are always the same, no matter what the position is. This is useful when |
||||
* simulating a partly-compressible stream of user data. |
||||
*/ |
||||
public class UniformRandomPayloadGenerator implements PayloadGenerator { |
||||
private final int size; |
||||
private final long seed; |
||||
private final int padding; |
||||
private final Random random = new Random(); |
||||
private final byte[] padBytes; |
||||
private final byte[] randomBytes; |
||||
|
||||
@JsonCreator |
||||
public UniformRandomPayloadGenerator(@JsonProperty("size") int size, |
||||
@JsonProperty("seed") long seed, |
||||
@JsonProperty("padding") int padding) { |
||||
this.size = size; |
||||
this.seed = seed; |
||||
this.padding = padding; |
||||
if (padding < 0 || padding > size) { |
||||
throw new RuntimeException("Invalid value " + padding + " for " + |
||||
"padding: the number of padding bytes must not be smaller than " + |
||||
"0 or greater than the total payload size."); |
||||
} |
||||
this.padBytes = new byte[padding]; |
||||
random.setSeed(seed); |
||||
random.nextBytes(padBytes); |
||||
this.randomBytes = new byte[size - padding]; |
||||
} |
||||
|
||||
@JsonProperty |
||||
public int size() { |
||||
return size; |
||||
} |
||||
|
||||
@JsonProperty |
||||
public long seed() { |
||||
return seed; |
||||
} |
||||
|
||||
@JsonProperty |
||||
public int padding() { |
||||
return padding; |
||||
} |
||||
|
||||
@Override |
||||
public synchronized byte[] generate(long position) { |
||||
byte[] result = new byte[size]; |
||||
if (randomBytes.length > 0) { |
||||
random.setSeed(seed + position); |
||||
random.nextBytes(randomBytes); |
||||
System.arraycopy(randomBytes, 0, result, 0, Math.min(randomBytes.length, result.length)); |
||||
} |
||||
if (padBytes.length > 0) { |
||||
System.arraycopy(padBytes, 0, result, randomBytes.length, result.length - randomBytes.length); |
||||
} |
||||
return result; |
||||
} |
||||
} |
Loading…
Reference in new issue