diff --git a/bin/m-client b/bin/m-client index c106a26..a557a4e 100755 --- a/bin/m-client +++ b/bin/m-client @@ -30,6 +30,7 @@ CLIENT_CONSOLECONSUMER_CLASS="com.inmobi.messaging.consumer.examples.ConsoleClie CLIENT_BENCHMARK_CLASS="com.inmobi.messaging.consumer.examples.StreamingBenchmark" CLIENT_COUNTER_CLASS="com.inmobi.messaging.consumer.examples.CounterClient" CLIENT_MULTITOPIC_SEQGEN_CLASS="com.inmobi.messaging.publisher.examples.MultiTopicSeqGenerator" +CLIENT_CHECKPOINT_UTIL_CLASS="com.inmobi.messaging.consumer.util.CheckpointUtil" ################################ # functions @@ -69,7 +70,7 @@ Usage: $0 file --conf [-consumer [ ]] --conf $0 counter --conf $0 multitopicseqgen --conf - + $0 checkpointutil EOF } @@ -125,6 +126,9 @@ case "$mode" in multitopicseqgen) opt_multitopicseqgen=1 ;; + checkpointutil) + opt_checkpointutil=1 + ;; *) error "Unknown or unspecified command '$mode'" echo @@ -204,8 +208,9 @@ elif [ -n "$opt_counter" ] ; then run_client $CLIENT_COUNTER_CLASS $args elif [ -n "$opt_multitopicseqgen" ] ; then run_client $CLIENT_MULTITOPIC_SEQGEN_CLASS $args +elif [ -n "$opt_checkpointutil" ] ; then + run_client $CLIENT_CHECKPOINT_UTIL_CLASS $args else error "This message should never appear" 1 fi - exit 0 diff --git a/messaging-client-core/src/main/java/com/inmobi/messaging/consumer/AbstractMessageConsumer.java b/messaging-client-core/src/main/java/com/inmobi/messaging/consumer/AbstractMessageConsumer.java index 9bb8afa..1b5bdb7 100644 --- a/messaging-client-core/src/main/java/com/inmobi/messaging/consumer/AbstractMessageConsumer.java +++ b/messaging-client-core/src/main/java/com/inmobi/messaging/consumer/AbstractMessageConsumer.java @@ -95,6 +95,7 @@ public void init(String topicName, String consumerName, Date startTimestamp, startTime.after(new Date(System.currentTimeMillis()))) { throw new IllegalArgumentException("Future start time is not accepted"); } + init(config); metrics = (BaseMessageConsumerStatsExposer) getMetricsImpl(); String emitterConfig = config .getString(MessageConsumerFactory.EMITTER_CONF_FILE_KEY); @@ -102,7 +103,6 @@ public void init(String topicName, String consumerName, Date startTimestamp, statsEmitter.init(emitterConfig); statsEmitter.add(metrics); } - init(config); } /** diff --git a/messaging-client-databus/src/main/java/com/inmobi/databus/files/FileMap.java b/messaging-client-databus/src/main/java/com/inmobi/databus/files/FileMap.java index 83bb73d..81cb2ae 100644 --- a/messaging-client-databus/src/main/java/com/inmobi/databus/files/FileMap.java +++ b/messaging-client-databus/src/main/java/com/inmobi/databus/files/FileMap.java @@ -98,6 +98,14 @@ public FileStatus getFirstFile() { return null; } + public FileStatus getLastFile() { + Map.Entry last = files.lastEntry(); + if (last != null) { + return last.getValue(); + } + return null; + } + private Map.Entry getFirstEntry() { return files.firstEntry(); } @@ -156,5 +164,4 @@ public FileStatus getNext() { public boolean hasNext() { return fileNameIterator.hasNext(); } - } diff --git a/messaging-client-databus/src/main/java/com/inmobi/databus/partition/ClusterReader.java b/messaging-client-databus/src/main/java/com/inmobi/databus/partition/ClusterReader.java index fabbca2..6a5e7d9 100644 --- a/messaging-client-databus/src/main/java/com/inmobi/databus/partition/ClusterReader.java +++ b/messaging-client-databus/src/main/java/com/inmobi/databus/partition/ClusterReader.java @@ -4,6 +4,10 @@ import java.io.DataInputStream; import java.io.IOException; import java.util.Date; +import java.util.Iterator; +import java.util.Map; +import java.util.Collection; +import java.util.Set; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -13,34 +17,77 @@ import org.apache.hadoop.io.Text; import com.inmobi.databus.readers.DatabusStreamWaitingReader; +import com.inmobi.messaging.consumer.databus.MessageCheckpoint; import com.inmobi.messaging.metrics.PartitionReaderStatsExposer; public class ClusterReader extends AbstractPartitionStreamReader { private static final Log LOG = LogFactory.getLog(PartitionReader.class); - private final PartitionCheckpoint partitionCheckpoint; + private final PartitionCheckpointList partitionCheckpointList; private final Date startTime; private final Path streamDir; private final boolean isDatabusData; ClusterReader(PartitionId partitionId, - PartitionCheckpoint partitionCheckpoint, FileSystem fs, + PartitionCheckpointList partitionCheckpointList, FileSystem fs, Path streamDir, Configuration conf, String inputFormatClass, Date startTime, long waitTimeForFileCreate, boolean isDatabusData, - PartitionReaderStatsExposer metrics, boolean noNewFiles) + PartitionReaderStatsExposer metrics, boolean noNewFiles, + Set partitionMinList) throws IOException { this.startTime = startTime; this.streamDir = streamDir; - this.partitionCheckpoint = partitionCheckpoint; + this.partitionCheckpointList = partitionCheckpointList; this.isDatabusData = isDatabusData; reader = new DatabusStreamWaitingReader(partitionId, fs, streamDir, - inputFormatClass, conf, waitTimeForFileCreate, metrics, noNewFiles); + inputFormatClass, conf, waitTimeForFileCreate, metrics, noNewFiles, + partitionMinList, partitionCheckpointList); } + + /* + + * this method is used to find the partition checkpoint which has least time stamp. + + * So that reader starts build listing from this partition checkpoint time stamp). + + */ + public PartitionCheckpoint findLeastPartitionCheckPointTime( + PartitionCheckpointList partitionCheckpointList) { + PartitionCheckpoint partitioncheckpoint = null; + + Map listOfCheckpoints = + partitionCheckpointList.getCheckpoints(); + + if (listOfCheckpoints != null) { + Collection listofPartitionCheckpoints = + listOfCheckpoints.values(); + Iterator it = listofPartitionCheckpoints.iterator(); + Date timeStamp = null; + if (it.hasNext()) { + partitioncheckpoint = it.next(); + timeStamp = DatabusStreamWaitingReader.getDateFromStreamDir(streamDir, + new Path(partitioncheckpoint.getFileName())); + } + while (it.hasNext()) { + PartitionCheckpoint tmpPartitionCheckpoint = it.next(); + Date date = DatabusStreamWaitingReader.getDateFromStreamDir(streamDir, + new Path(tmpPartitionCheckpoint.getFileName())); + if (timeStamp.compareTo(date) > 0) { + partitioncheckpoint = tmpPartitionCheckpoint; + timeStamp = date; + } + } + } + return partitioncheckpoint; + } + public void initializeCurrentFile() throws IOException, InterruptedException { LOG.info("Initializing partition reader's current file"); + PartitionCheckpoint partitionCheckpoint = null; + if (partitionCheckpointList != null) { + partitionCheckpoint = findLeastPartitionCheckPointTime(partitionCheckpointList); + } + if (startTime != null) { ((DatabusStreamWaitingReader)reader).build(startTime); if (!reader.initializeCurrentFile(startTime)) { @@ -52,9 +99,12 @@ public void initializeCurrentFile() throws IOException, InterruptedException { DatabusStreamWaitingReader.getBuildTimestamp(streamDir, partitionCheckpoint)); if (!reader.isEmpty()) { - if (!reader.initializeCurrentFile(partitionCheckpoint)) { - throw new IllegalArgumentException("Checkpoint file does not exist"); - } + if (partitionCheckpoint.getLineNum() == -1) { + reader.initFromNextCheckPoint(); + } + else if (!reader.initializeCurrentFile(partitionCheckpoint)) { + throw new IllegalArgumentException("Checkpoint file does not exist"); + } } else { reader.startFromBegining(); } @@ -75,4 +125,16 @@ public byte[] readLine() throws IOException, InterruptedException { } return line; } + + @Override + public MessageCheckpoint getMessageCheckpoint() { + if (reader instanceof DatabusStreamWaitingReader) { + DatabusStreamWaitingReader dataWaitingReader = (DatabusStreamWaitingReader) reader; + PartitionCheckpointList pChkLst = new PartitionCheckpointList( + dataWaitingReader.getPartitionCheckpointList().getCheckpoints()); + return pChkLst; + } else { + return null; + } + } } diff --git a/messaging-client-databus/src/main/java/com/inmobi/databus/partition/CollectorReader.java b/messaging-client-databus/src/main/java/com/inmobi/databus/partition/CollectorReader.java index 8f4659f..c9a28aa 100644 --- a/messaging-client-databus/src/main/java/com/inmobi/databus/partition/CollectorReader.java +++ b/messaging-client-databus/src/main/java/com/inmobi/databus/partition/CollectorReader.java @@ -15,6 +15,7 @@ import com.inmobi.databus.files.DatabusStreamFile; import com.inmobi.databus.readers.CollectorStreamReader; import com.inmobi.databus.readers.LocalStreamCollectorReader; +import com.inmobi.messaging.consumer.databus.MessageCheckpoint; import com.inmobi.messaging.metrics.CollectorReaderStatsExposer; public class CollectorReader extends AbstractPartitionStreamReader { @@ -173,5 +174,10 @@ public byte[] readLine() throws IOException, InterruptedException { } return line; } - + + @Override + public MessageCheckpoint getMessageCheckpoint() { + return new PartitionCheckpoint(reader.getCurrentStreamFile(), + reader.getCurrentLineNum()); + } } diff --git a/messaging-client-databus/src/main/java/com/inmobi/databus/partition/PartitionCheckpoint.java b/messaging-client-databus/src/main/java/com/inmobi/databus/partition/PartitionCheckpoint.java index e6b8be2..36eaad0 100644 --- a/messaging-client-databus/src/main/java/com/inmobi/databus/partition/PartitionCheckpoint.java +++ b/messaging-client-databus/src/main/java/com/inmobi/databus/partition/PartitionCheckpoint.java @@ -7,8 +7,9 @@ import org.apache.hadoop.io.Writable; import com.inmobi.databus.files.StreamFile; +import com.inmobi.messaging.consumer.databus.MessageCheckpoint; -public class PartitionCheckpoint implements Writable { +public class PartitionCheckpoint implements Writable, MessageCheckpoint { private StreamFile streamFile; private long lineNum; diff --git a/messaging-client-databus/src/main/java/com/inmobi/databus/partition/PartitionCheckpointList.java b/messaging-client-databus/src/main/java/com/inmobi/databus/partition/PartitionCheckpointList.java new file mode 100644 index 0000000..1cfbd35 --- /dev/null +++ b/messaging-client-databus/src/main/java/com/inmobi/databus/partition/PartitionCheckpointList.java @@ -0,0 +1,51 @@ +package com.inmobi.databus.partition; + +import java.util.Map; +import java.util.TreeMap; + +import com.inmobi.databus.partition.PartitionCheckpoint; +import com.inmobi.messaging.consumer.databus.MessageCheckpoint; + +/** + * Checkpoint for the segments of databus stream consumer. + * + */ +public class PartitionCheckpointList implements MessageCheckpoint { + + // map of static id to its checkpoint + private Map pChkpoints = + new TreeMap(); + + public PartitionCheckpointList(Map chkpoints) { + + this.pChkpoints = chkpoints; + } + + public void setCheckpoint(Map chkpoints) { + this.pChkpoints = chkpoints; + } + + public Map getCheckpoints() { + return pChkpoints; + } + + public void set(int segmentId, PartitionCheckpoint pck) { + pChkpoints.put(segmentId, pck); + } + + public String toString() { + StringBuffer buf = new StringBuffer(); + for (Map.Entry entry : pChkpoints + .entrySet()) { + buf.append(entry.getKey().toString()) + .append(":"); + if (entry.getValue() != null) { + buf.append(entry.getValue().toString()); + } else { + buf.append("null"); + } + buf.append(", "); + } + return buf.toString(); + } +} diff --git a/messaging-client-databus/src/main/java/com/inmobi/databus/partition/PartitionReader.java b/messaging-client-databus/src/main/java/com/inmobi/databus/partition/PartitionReader.java index 07ec9d2..7e6239d 100644 --- a/messaging-client-databus/src/main/java/com/inmobi/databus/partition/PartitionReader.java +++ b/messaging-client-databus/src/main/java/com/inmobi/databus/partition/PartitionReader.java @@ -3,6 +3,7 @@ import java.io.IOException; import java.nio.ByteBuffer; import java.util.Date; +import java.util.Set; import java.util.concurrent.BlockingQueue; import org.apache.commons.codec.binary.Base64; @@ -15,6 +16,7 @@ import com.inmobi.databus.files.StreamFile; import com.inmobi.messaging.Message; import com.inmobi.messaging.consumer.databus.DataEncodingType; +import com.inmobi.messaging.consumer.databus.MessageCheckpoint; import com.inmobi.messaging.consumer.databus.QueueEntry; import com.inmobi.messaging.metrics.CollectorReaderStatsExposer; import com.inmobi.messaging.metrics.PartitionReaderStatsExposer; @@ -47,15 +49,16 @@ public PartitionReader(PartitionId partitionId, } public PartitionReader(PartitionId partitionId, - PartitionCheckpoint partitionCheckpoint, FileSystem fs, - BlockingQueue buffer, Path streamDir, + PartitionCheckpointList partitionCheckpointList, FileSystem fs, + BlockingQueue buffer, Path streamDir, Configuration conf, String inputFormatClass, Date startTime, long waitTimeForFileCreate, boolean isDatabusData, - DataEncodingType dataEncoding, PartitionReaderStatsExposer prMetrics) + DataEncodingType dataEncoding, PartitionReaderStatsExposer prMetrics, + Set partitionMinList) throws IOException { - this(partitionId, partitionCheckpoint, fs, buffer, streamDir, + this(partitionId, partitionCheckpointList, fs, buffer, streamDir, conf, inputFormatClass, startTime, waitTimeForFileCreate, isDatabusData, - dataEncoding, prMetrics, false); + dataEncoding, prMetrics, false, partitionMinList); } PartitionReader(PartitionId partitionId, @@ -82,21 +85,22 @@ public PartitionReader(PartitionId partitionId, } PartitionReader(PartitionId partitionId, - PartitionCheckpoint partitionCheckpoint, FileSystem fs, + PartitionCheckpointList partitionCheckpointList, FileSystem fs, BlockingQueue buffer, Path streamDir, Configuration conf, String inputFormatClass, Date startTime, long waitTimeForFileCreate, boolean isDatabusData, DataEncodingType dataEncoding, PartitionReaderStatsExposer prMetrics, - boolean noNewFiles) + boolean noNewFiles, Set partitionMinList) throws IOException { - this(partitionId, partitionCheckpoint, buffer, startTime, dataEncoding, - prMetrics); - reader = new ClusterReader(partitionId, partitionCheckpoint, + this(partitionId, partitionCheckpointList, buffer, startTime, dataEncoding, + prMetrics, partitionMinList); + reader = new ClusterReader(partitionId, partitionCheckpointList, fs, streamDir, conf, inputFormatClass, startTime, - waitTimeForFileCreate, isDatabusData, prMetrics, noNewFiles); + waitTimeForFileCreate, isDatabusData, prMetrics, noNewFiles, + partitionMinList); // initialize cluster and its directories LOG.info("Partition reader initialized with partitionId:" + partitionId + - " checkPoint:" + partitionCheckpoint + + " checkPoint:" + partitionCheckpointList + " startTime:" + startTime + " currentReader:" + reader); } @@ -118,6 +122,24 @@ private PartitionReader(PartitionId partitionId, this.dataEncoding = dataEncoding; this.prMetrics = prMetrics; } + + private PartitionReader(PartitionId partitionId, + PartitionCheckpointList partitionCheckpointList, + BlockingQueue buffer, Date startTime, + DataEncodingType dataEncoding, + PartitionReaderStatsExposer prMetrics, Set partitionMinList) + throws IOException { + if (startTime == null && partitionCheckpointList == null) { + String msg = "StartTime and checkpoint both" + + " cannot be null in PartitionReader"; + LOG.warn(msg); + throw new IllegalArgumentException(msg); + } + this.partitionId = partitionId; + this.buffer = buffer; + this.dataEncoding = dataEncoding; + this.prMetrics = prMetrics; + } public synchronized void start() { Runnable runnable = new Runnable() { @@ -210,10 +232,9 @@ void execute() { } else { data = line; } + MessageCheckpoint checkpoint = reader.getMessageCheckpoint(); buffer.put(new QueueEntry(new Message( - ByteBuffer.wrap(data)), partitionId, - new PartitionCheckpoint(reader.getCurrentFile(), - reader.getCurrentLineNum()))); + ByteBuffer.wrap(data)), partitionId, checkpoint)); prMetrics.incrementMessagesAddedToBuffer(); } else { LOG.info("No stream to read"); diff --git a/messaging-client-databus/src/main/java/com/inmobi/databus/partition/PartitionStreamReader.java b/messaging-client-databus/src/main/java/com/inmobi/databus/partition/PartitionStreamReader.java index a8ea302..77d7b05 100644 --- a/messaging-client-databus/src/main/java/com/inmobi/databus/partition/PartitionStreamReader.java +++ b/messaging-client-databus/src/main/java/com/inmobi/databus/partition/PartitionStreamReader.java @@ -3,6 +3,7 @@ import java.io.IOException; import com.inmobi.databus.files.StreamFile; +import com.inmobi.messaging.consumer.databus.MessageCheckpoint; public interface PartitionStreamReader { @@ -19,4 +20,6 @@ public interface PartitionStreamReader { void closeStream() throws IOException; void close() throws IOException; + + MessageCheckpoint getMessageCheckpoint(); } diff --git a/messaging-client-databus/src/main/java/com/inmobi/databus/readers/CollectorStreamReader.java b/messaging-client-databus/src/main/java/com/inmobi/databus/readers/CollectorStreamReader.java index 7a5e9bd..ef895f1 100644 --- a/messaging-client-databus/src/main/java/com/inmobi/databus/readers/CollectorStreamReader.java +++ b/messaging-client-databus/src/main/java/com/inmobi/databus/readers/CollectorStreamReader.java @@ -198,8 +198,8 @@ public byte[] readLine() throws IOException, InterruptedException { } } else { if (moveToNext) { - setNextFile(); - LOG.info("Reading from next file: " + getCurrentFile()); + setNextFile(); + LOG.info("Reading from next file: " + getCurrentFile()); } else { LOG.info("Reading from same file before moving to next"); // open the same file diff --git a/messaging-client-databus/src/main/java/com/inmobi/databus/readers/DatabusStreamWaitingReader.java b/messaging-client-databus/src/main/java/com/inmobi/databus/readers/DatabusStreamWaitingReader.java index 2aeba3c..f639283 100644 --- a/messaging-client-databus/src/main/java/com/inmobi/databus/readers/DatabusStreamWaitingReader.java +++ b/messaging-client-databus/src/main/java/com/inmobi/databus/readers/DatabusStreamWaitingReader.java @@ -3,6 +3,8 @@ import java.io.IOException; import java.util.Calendar; import java.util.Date; +import java.util.Map; +import java.util.Set; import java.util.TreeMap; import org.apache.commons.logging.Log; @@ -16,6 +18,7 @@ import com.inmobi.databus.files.FileMap; import com.inmobi.databus.files.HadoopStreamFile; import com.inmobi.databus.partition.PartitionCheckpoint; +import com.inmobi.databus.partition.PartitionCheckpointList; import com.inmobi.databus.partition.PartitionId; import com.inmobi.messaging.metrics.PartitionReaderStatsExposer; @@ -25,16 +28,79 @@ public class DatabusStreamWaitingReader private static final Log LOG = LogFactory.getLog( DatabusStreamWaitingReader.class); + public int currentMin; + public Set partitionMinList; + public PartitionCheckpointList partitionCheckpointList; + public DatabusStreamWaitingReader(PartitionId partitionId, FileSystem fs, Path streamDir, String inputFormatClass, Configuration conf, long waitTimeForFileCreate, PartitionReaderStatsExposer metrics, - boolean noNewFiles) + boolean noNewFiles, Set partitionMinList, + PartitionCheckpointList partitionCheckpointList) throws IOException { super(partitionId, fs, streamDir, inputFormatClass, conf, waitTimeForFileCreate, metrics, noNewFiles); + this.partitionCheckpointList = partitionCheckpointList; + this.partitionMinList = partitionMinList; + } + + public boolean isRead(Date currentTimeStamp, int minute) { + PartitionCheckpoint partitionCheckpoint = null; + if (partitionCheckpointList != null && (partitionCheckpointList. + getCheckpoints() != null)) { + partitionCheckpoint = partitionCheckpointList.getCheckpoints().get(minute); + if (partitionCheckpoint != null) { + Date checkpointedTimestamp = getDateFromStreamDir(streamDir, + new Path(partitionCheckpoint.getFileName())); + if ((currentTimeStamp.compareTo(checkpointedTimestamp) < 0) || + (partitionCheckpoint.getLineNum() == -1)) { + return true; + } + } + } + return false; + } + + public PartitionCheckpointList getPartitionCheckpointList() { + TreeMap chckMp = new TreeMap(); + Map orginalMap = partitionCheckpointList. + getCheckpoints(); + for(Map.Entry entry: orginalMap.entrySet()) { + chckMp.put(entry.getKey(), new PartitionCheckpoint(entry.getValue(). + getStreamFile(), entry.getValue().getLineNum())); + } + return new PartitionCheckpointList(chckMp); } + + + @Override + public boolean initFromNextCheckPoint() throws IOException { + initCurrentFile(); + currentFile = getFirstFileInStream(); + Date date = DatabusStreamWaitingReader.getDateFromStreamDir(streamDir, + currentFile.getPath().getParent()); + int currentMinute = date.getMinutes(); + PartitionCheckpoint partitioncheckpoint = partitionCheckpointList. + getCheckpoints().get(currentMinute); + if (partitioncheckpoint != null) { + if (currentFile.getPath().getName().compareTo( + partitioncheckpoint.getFileName()) != 0) { + currentFile = fs.getFileStatus(new Path(partitioncheckpoint.getFileName())); + setIteratorToFile(currentFile); + } + currentLineNum = partitioncheckpoint.getLineNum(); + } + if (currentFile != null) { + LOG.debug("CurrentFile:" + getCurrentFile() + " currentLineNum:" + + currentLineNum); + setIterator(); + } + return currentFile != null; - protected void buildListing(FileMap fmap, + } + + protected void buildListing(FileMap fmap, PathFilter pathFilter) throws IOException { Calendar current = Calendar.getInstance(); @@ -48,12 +114,18 @@ protected void buildListing(FileMap fmap, while (current.getTime().before(now) && hour == current.get(Calendar.HOUR_OF_DAY)) { Path dir = getMinuteDirPath(streamDir, current.getTime()); + int min = current.get(Calendar.MINUTE); + Date currenTimestamp = current.getTime(); current.add(Calendar.MINUTE, 1); if (fs.exists(dir)) { // Move the current minute to next minute Path nextMinDir = getMinuteDirPath(streamDir, current.getTime()); if (fs.exists(nextMinDir)) { - doRecursiveListing(dir, pathFilter, fmap); + if (partitionMinList.contains(new Integer(min))) { + if (!isRead(currenTimestamp, min)) { + doRecursiveListing(dir, pathFilter, fmap); + } + } } else { LOG.info("Reached end of file listing. Not looking at the last" + " minute directory:" + dir); @@ -73,6 +145,39 @@ protected void buildListing(FileMap fmap, } } } + + public boolean prepareMoveToNext(FileStatus currentFile, FileStatus nextFile) + throws IOException { + Date date = getDateFromStreamDir(streamDir, currentFile.getPath().getParent()); + Calendar now = Calendar.getInstance(); + now.setTime(date); + currentMin = now.get(Calendar.MINUTE); + + date = getDateFromStreamDir(streamDir, nextFile.getPath().getParent()); + now.setTime(date); + + if (currentMin != now.get(Calendar.MINUTE)) { + partitionCheckpointList.set(currentMin, + new PartitionCheckpoint(getCurrentStreamFile(), -1)); + currentMin = now.get(Calendar.MINUTE); + PartitionCheckpoint partitionCheckpoint = partitionCheckpointList. + getCheckpoints().get(currentMin); + if (partitionCheckpoint != null && partitionCheckpoint.getLineNum() != -1) { + currentFile = nextFile; + //set iterator to checkpoointed file if there is a checkpoint + if((partitionCheckpoint.getStreamFile()).compareTo( + getStreamFile(currentFile)) != 0) { + currentFile = fs.getFileStatus(new Path(partitionCheckpoint. + getFileName())); + setIteratorToFile(currentFile); + } + currentLineNum = partitionCheckpoint.getLineNum(); + nextFile = currentFile; + return false; + } + } + return true; + } @Override protected HadoopStreamFile getStreamFile(Date timestamp) { @@ -107,6 +212,7 @@ private void waitForNextFileCreation(FileStatus file) @Override public byte[] readLine() throws IOException, InterruptedException { byte[] line = readNextLine(); + currentMin = getDateFromStreamDir(streamDir, getCurrentFile()).getMinutes(); while (line == null) { // reached end of file LOG.info("Read " + getCurrentFile() + " with lines:" + currentLineNum); if (closed) { @@ -134,6 +240,10 @@ public byte[] readLine() throws IOException, InterruptedException { } line = readNextLine(); } + if (partitionMinList.contains(currentMin)) { + partitionCheckpointList.set(currentMin, new PartitionCheckpoint( + getCurrentStreamFile(), getCurrentLineNum())); + } return line; } diff --git a/messaging-client-databus/src/main/java/com/inmobi/databus/readers/StreamReader.java b/messaging-client-databus/src/main/java/com/inmobi/databus/readers/StreamReader.java index f369d46..7251cdb 100644 --- a/messaging-client-databus/src/main/java/com/inmobi/databus/readers/StreamReader.java +++ b/messaging-client-databus/src/main/java/com/inmobi/databus/readers/StreamReader.java @@ -44,6 +44,16 @@ protected StreamReader(PartitionId partitionId, FileSystem fs, this.noNewFiles = noNewFiles; this.fileMap = createFileMap(); } + + public boolean initFromNextCheckPoint() throws IOException { + //The method is overridden in the DatabusStreamWaitingReader + return true; + } + + public boolean prepareMoveToNext(FileStatus currentFile, FileStatus nextFile) + throws IOException { + return true; + } public void openStream() throws IOException { openCurrentFile(false); @@ -225,17 +235,18 @@ protected void resetCurrentFileSettings() { protected boolean nextFile() throws IOException { if (hasNextFile()) { - setNextFile(); - return true; + setNextFile(); + return true; } return false; } protected void setNextFile() throws IOException { - FileStatus nextFile = fileMap.getNext(); + FileStatus nextFile = fileMap.getNext(); if (nextFile != null) { + boolean next = prepareMoveToNext(currentFile, nextFile); currentFile = nextFile; - openCurrentFile(true); + openCurrentFile(next); } } @@ -328,4 +339,8 @@ public boolean isBeforeStream(String fileName) throws IOException { protected boolean isWithinStream(String fileName) throws IOException { return fileMap.isWithin(fileName); } + + protected FileStatus getFirstFileInStream() { + return fileMap.getFirstFile(); + } } diff --git a/messaging-client-databus/src/main/java/com/inmobi/messaging/consumer/databus/AbstractMessagingDatabusConsumer.java b/messaging-client-databus/src/main/java/com/inmobi/messaging/consumer/databus/AbstractMessagingDatabusConsumer.java index 94836cc..0bb3146 100644 --- a/messaging-client-databus/src/main/java/com/inmobi/messaging/consumer/databus/AbstractMessagingDatabusConsumer.java +++ b/messaging-client-databus/src/main/java/com/inmobi/messaging/consumer/databus/AbstractMessagingDatabusConsumer.java @@ -4,7 +4,9 @@ import java.lang.reflect.Constructor; import java.util.Date; import java.util.HashMap; +import java.util.HashSet; import java.util.Map; +import java.util.Set; import java.util.concurrent.BlockingQueue; import java.util.concurrent.LinkedBlockingQueue; @@ -14,6 +16,7 @@ import com.inmobi.databus.CheckpointProvider; import com.inmobi.databus.partition.PartitionCheckpoint; +import com.inmobi.databus.partition.PartitionCheckpointList; import com.inmobi.databus.partition.PartitionId; import com.inmobi.databus.partition.PartitionReader; import com.inmobi.databus.utils.SecureLoginUtil; @@ -21,7 +24,7 @@ import com.inmobi.messaging.ClientConfig; import com.inmobi.messaging.Message; import com.inmobi.messaging.consumer.AbstractMessageConsumer; -import com.inmobi.messaging.consumer.BaseMessageConsumerStatsExposer; +import com.inmobi.messaging.metrics.DatabusConsumerStatsExposer; public abstract class AbstractMessagingDatabusConsumer extends AbstractMessageConsumer @@ -36,12 +39,14 @@ public abstract class AbstractMessagingDatabusConsumer new HashMap(); protected CheckpointProvider checkpointProvider; - protected Checkpoint currentCheckpoint; + protected ConsumerCheckpoint currentCheckpoint; protected long waitTimeForFileCreate; protected int bufferSize; protected DataEncodingType dataEncodingType; protected int retentionInHours; - + protected int consumerNumber; + protected int totalConsumers; + public Set partitionMinList; @Override protected void init(ClientConfig config) throws IOException { initializeConfig(config); @@ -78,6 +83,27 @@ protected void initializeConfig(ClientConfig config) throws IOException { } } + // Read consumer id + String consumerIdStr = config.getString(consumerIdInGroupConfig, + DEFAULT_CONSUMER_ID); + String[] id = consumerIdStr.split("/"); + try { + consumerNumber = Integer.parseInt(id[0]); + totalConsumers = Integer.parseInt(id[1]); + partitionMinList = new HashSet(); + if (consumerNumber > 0 && totalConsumers > 0) { + for (int i = 0; i < 60; i++) { + if ((i % totalConsumers) == (consumerNumber - 1)) { + partitionMinList.add(i); + } + } + } else { + LOG.info("Invalid consumer group membership"); + } + } catch (NumberFormatException nfe) { + throw new IllegalArgumentException("Invalid consumer group membership", + nfe); + } // Create checkpoint provider and initialize checkpoint String chkpointProviderClassName = config.getString( chkProviderConfig, DEFAULT_CHK_PROVIDER); @@ -86,14 +112,8 @@ protected void initializeConfig(ClientConfig config) throws IOException { this.checkpointProvider = createCheckpointProvider( chkpointProviderClassName, databusCheckpointDir); - byte[] chkpointData = checkpointProvider.read(getChkpointKey()); - if (chkpointData != null) { - this.currentCheckpoint = new Checkpoint(chkpointData); - } else { - Map partitionsChkPoints = - new HashMap(); - this.currentCheckpoint = new Checkpoint(partitionsChkPoints); - } + createCheckpoint(); + currentCheckpoint.read(checkpointProvider, getChkpointKey()); //create buffer bufferSize = config.getInteger(queueSizeConfig, DEFAULT_QUEUE_SIZE); @@ -116,15 +136,22 @@ public Map getPartitionReaders() { return readers; } - public Checkpoint getCurrentCheckpoint() { - return currentCheckpoint; + protected abstract void createCheckpoint(); + + public Set getPartitionMinList() { + return partitionMinList; + } + + public ConsumerCheckpoint getCurrentCheckpoint() { + return currentCheckpoint; } @Override protected Message getNext() throws InterruptedException { QueueEntry entry; entry = buffer.take(); - currentCheckpoint.set(entry.getPartitionId(), entry.getPartitionChkpoint()); + MessageCheckpoint msgchk = entry.getMessageChkpoint(); + currentCheckpoint.set(entry.getPartitionId(), msgchk); return entry.getMessage(); } @@ -153,9 +180,42 @@ protected Date getPartitionTimestamp(PartitionId id, PartitionCheckpoint pck, LOG.info("Creating partition with timestamp: " + partitionTimestamp + " checkpoint:" + pck); } - return partitionTimestamp; } + + protected Date getPartitionTimestamp(PartitionId id, PartitionCheckpointList pck, + Date allowedStartTime) { + boolean checkpointFlag = isPartitionCheckpointListNUll(pck); + Date partitionTimestamp = startTime; + if (startTime == null && !checkpointFlag) { + LOG.info("There is no startTime passed and no checkpoint exists" + + " for the partition: " + id + " starting from the start" + + " of the stream."); + partitionTimestamp = allowedStartTime; + } else if (startTime != null && startTime.before(allowedStartTime)) { + LOG.info("Start time passed is before the start of the stream," + + " starting from the start of the stream."); + partitionTimestamp = allowedStartTime; + } else { + LOG.info("Creating partition with timestamp: " + partitionTimestamp + + " checkpoint:" + pck); + } + + return partitionTimestamp; + } + + public boolean isPartitionCheckpointListNUll(PartitionCheckpointList + partitionCheckpointList) { + MaplistOfPartitionCheckpoints = + partitionCheckpointList.getCheckpoints(); + for (Integer minuteId : partitionMinList) { + if ( listOfPartitionCheckpoints.get(minuteId)!= null) { + return true; + } + } + return false; + } + protected String getChkpointKey() { return consumerName + "_" + topicName; @@ -166,8 +226,7 @@ protected void doReset() throws IOException { // restart the service, consumer will start streaming from the last saved // checkpoint close(); - this.currentCheckpoint = new Checkpoint( - checkpointProvider.read(getChkpointKey())); + currentCheckpoint.read(checkpointProvider, getChkpointKey()); LOG.info("Resetting to checkpoint:" + currentCheckpoint); // reset to last marked position, ignore start time startTime = null; @@ -177,8 +236,7 @@ protected void doReset() throws IOException { @Override protected void doMark() throws IOException { - checkpointProvider.checkpoint(getChkpointKey(), - currentCheckpoint.toBytes()); + currentCheckpoint.write(checkpointProvider, getChkpointKey()); LOG.info("Committed checkpoint:" + currentCheckpoint); } @@ -200,6 +258,7 @@ public boolean isMarkSupported() { @Override protected AbstractMessagingClientStatsExposer getMetricsImpl() { - return new BaseMessageConsumerStatsExposer(topicName, consumerName); + return new DatabusConsumerStatsExposer(topicName, consumerName, + consumerNumber); } } diff --git a/messaging-client-databus/src/main/java/com/inmobi/messaging/consumer/databus/Checkpoint.java b/messaging-client-databus/src/main/java/com/inmobi/messaging/consumer/databus/Checkpoint.java index e71b147..9778128 100644 --- a/messaging-client-databus/src/main/java/com/inmobi/messaging/consumer/databus/Checkpoint.java +++ b/messaging-client-databus/src/main/java/com/inmobi/messaging/consumer/databus/Checkpoint.java @@ -12,6 +12,7 @@ import org.apache.hadoop.io.Writable; +import com.inmobi.databus.CheckpointProvider; import com.inmobi.databus.partition.PartitionCheckpoint; import com.inmobi.databus.partition.PartitionId; @@ -21,7 +22,7 @@ * It holds checkpoint for all the partitions. * */ -public class Checkpoint implements Writable { +public class Checkpoint implements Writable, ConsumerCheckpoint { // map of partitionId to partition private Map partitionsChkPoint = @@ -45,10 +46,30 @@ public byte[] toBytes() throws IOException { public Map getPartitionsCheckpoint() { return partitionsChkPoint; } + + public void set(PartitionId partitionId, MessageCheckpoint partCheckpoint) { + this.set(partitionId, (PartitionCheckpoint)partCheckpoint); + } void set(PartitionId partitionId, PartitionCheckpoint partCheckpoint) { partitionsChkPoint.put(partitionId, partCheckpoint); } + + @Override + public void read(CheckpointProvider checkpointProvider, String key) + throws IOException { + byte[] chkpointData = checkpointProvider.read(key); + if (chkpointData != null) { + readFields(new DataInputStream(new ByteArrayInputStream(chkpointData))); + } + } + + @Override + public void write(CheckpointProvider checkpointProvider, String key) + throws IOException { + checkpointProvider.checkpoint(key, this.toBytes()); + } + @Override public void readFields(DataInput in) throws IOException { diff --git a/messaging-client-databus/src/main/java/com/inmobi/messaging/consumer/databus/CheckpointList.java b/messaging-client-databus/src/main/java/com/inmobi/messaging/consumer/databus/CheckpointList.java new file mode 100644 index 0000000..8233ec3 --- /dev/null +++ b/messaging-client-databus/src/main/java/com/inmobi/messaging/consumer/databus/CheckpointList.java @@ -0,0 +1,114 @@ +package com.inmobi.messaging.consumer.databus; + +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; +import java.util.Set; +import java.util.TreeMap; + +import org.mortbay.log.Log; + +import com.inmobi.databus.CheckpointProvider; +import com.inmobi.databus.partition.PartitionCheckpoint; +import com.inmobi.databus.partition.PartitionCheckpointList; +import com.inmobi.databus.partition.PartitionId; + +/** + * Checkpoint for the segments of databus stream consumer. + * + */ +public class CheckpointList implements ConsumerCheckpoint { + + // map of static id to its checkpoint + private Map chkpoints = + new TreeMap(); + private final Set idList; + + public CheckpointList(Set idList) { + this.idList = idList; + } + + void setCheckpoint(Map chkpoints) { + this.chkpoints = chkpoints; + } + + public Map getCheckpoints() { + return chkpoints; + } + + public void set(PartitionId pid, MessageCheckpoint msgCkp) { + PartitionCheckpointList pckList = (PartitionCheckpointList) msgCkp; + for (Map.Entry entry : pckList. + getCheckpoints().entrySet()) { + Checkpoint cp = chkpoints.get(entry.getKey()); + if (cp == null) { + Map partitionsChkPoints = + new HashMap(); + cp = new Checkpoint(partitionsChkPoints); + } + cp.set(pid, entry.getValue()); + chkpoints.put(entry.getKey(), cp); + } + } + + public String toString() { + StringBuffer buf = new StringBuffer(); + for (Map.Entry entry : chkpoints + .entrySet()) { + buf.append(entry.getKey().toString()) + .append(":"); + if (entry.getValue() != null) { + buf.append(entry.getValue().toString()); + } else { + buf.append("null"); + } + buf.append(", "); + } + return buf.toString(); + } + + public static String getChkpointKey(String superKey, int id) { + return superKey + "_" + id; + } + + public void write(CheckpointProvider checkpointProvider, String superKey) + throws IOException { + for (Map.Entry entry : chkpoints.entrySet()) { + checkpointProvider.checkpoint(getChkpointKey(superKey, entry.getKey()), + entry.getValue().toBytes()); + } + } + + public void preaprePartitionCheckPointList(PartitionId pid, + PartitionCheckpointList partitionCheckpointList) { + PartitionCheckpoint partitionCheckpoint; + if (!this.getCheckpoints().isEmpty()) { + for (Map.Entry entry : this.getCheckpoints().entrySet()) { + Checkpoint cp = entry.getValue(); + if (cp.getPartitionsCheckpoint().containsKey(pid)) { + partitionCheckpoint = cp.getPartitionsCheckpoint().get(pid); + partitionCheckpointList.set(entry.getKey(), partitionCheckpoint); + } + } + } + } + + public void read(CheckpointProvider checkpointProvider, String superKey) + throws IOException { + Map thisChkpoint = new TreeMap(); + for (Integer id : idList) { + byte[] chkpointData = checkpointProvider.read(getChkpointKey(superKey, id)); + Checkpoint checkpoint; + if (chkpointData != null) { + checkpoint = new Checkpoint(chkpointData); + } else { + Map partitionsChkPoints = + new HashMap(); + checkpoint = new Checkpoint(partitionsChkPoints); + } + Log.info("id" + id + "checkpoint" + checkpoint); + thisChkpoint.put(id, checkpoint); + } + setCheckpoint(thisChkpoint); + } +} diff --git a/messaging-client-databus/src/main/java/com/inmobi/messaging/consumer/databus/ConsumerCheckpoint.java b/messaging-client-databus/src/main/java/com/inmobi/messaging/consumer/databus/ConsumerCheckpoint.java new file mode 100644 index 0000000..07599d3 --- /dev/null +++ b/messaging-client-databus/src/main/java/com/inmobi/messaging/consumer/databus/ConsumerCheckpoint.java @@ -0,0 +1,16 @@ +package com.inmobi.messaging.consumer.databus; + +import java.io.IOException; + +import com.inmobi.databus.CheckpointProvider; +import com.inmobi.databus.partition.PartitionId; + +public interface ConsumerCheckpoint { + public void set(PartitionId pid, MessageCheckpoint pckList); + + public void read(CheckpointProvider checkpointProvider, String key) + throws IOException; + + public void write(CheckpointProvider checkpointProvider, String key) + throws IOException; +} diff --git a/messaging-client-databus/src/main/java/com/inmobi/messaging/consumer/databus/DatabusConsumer.java b/messaging-client-databus/src/main/java/com/inmobi/messaging/consumer/databus/DatabusConsumer.java index df079e9..8c48605 100644 --- a/messaging-client-databus/src/main/java/com/inmobi/messaging/consumer/databus/DatabusConsumer.java +++ b/messaging-client-databus/src/main/java/com/inmobi/messaging/consumer/databus/DatabusConsumer.java @@ -3,8 +3,10 @@ import java.io.IOException; import java.util.ArrayList; import java.util.Date; +import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.TreeMap; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -15,11 +17,10 @@ import org.apache.hadoop.mapred.TextInputFormat; import com.inmobi.databus.partition.PartitionCheckpoint; +import com.inmobi.databus.partition.PartitionCheckpointList; import com.inmobi.databus.partition.PartitionId; import com.inmobi.databus.partition.PartitionReader; -import com.inmobi.instrumentation.AbstractMessagingClientStatsExposer; import com.inmobi.messaging.ClientConfig; -import com.inmobi.messaging.Message; import com.inmobi.messaging.consumer.util.DatabusUtil; import com.inmobi.messaging.metrics.CollectorReaderStatsExposer; import com.inmobi.messaging.metrics.PartitionReaderStatsExposer; @@ -71,6 +72,10 @@ public class DatabusConsumer extends AbstractMessagingDatabusConsumer private static String clusterNamePrefix = "databusCluster"; protected void initializeConfig(ClientConfig config) throws IOException { + String type = config.getString(databusStreamType, DEFAULT_STREAM_TYPE); + streamType = StreamType.valueOf(type); + + super.initializeConfig(config); waitTimeForFlush = config.getLong(waitTimeForFlushConfig, DEFAULT_WAIT_TIME_FOR_FLUSH); @@ -88,9 +93,6 @@ protected void initializeConfig(ClientConfig config) throws IOException { for (int i = 0; i < rootDirSplits.length; i++) { rootDirs[i] = new Path(rootDirSplits[i]); } - String type = config.getString(databusStreamType, DEFAULT_STREAM_TYPE); - streamType = StreamType.valueOf(type); - if (streamType.equals(StreamType.MERGED)) { if (rootDirs.length > 1) { throw new IllegalArgumentException("Multiple directories are not" + @@ -119,8 +121,7 @@ private List getCollectors(FileSystem fs, Path baseDir) } protected void createPartitionReaders() throws IOException { - Map partitionsChkPoints = - currentCheckpoint.getPartitionsCheckpoint(); + // calculate the allowed start time long currentMillis = System.currentTimeMillis(); Date allowedStartTime = new Date(currentMillis - @@ -133,6 +134,8 @@ protected void createPartitionReaders() throws IOException { topicName); String clusterName = clusterNamePrefix + i; if (streamType.equals(StreamType.COLLECTOR)) { + Map partitionsChkPoints = + ((Checkpoint)currentCheckpoint).getPartitionsCheckpoint(); LOG.info("Creating partition readers for all the collectors"); for (String collector : getCollectors(fs, streamDir)) { PartitionId id = new PartitionId(clusterName, collector); @@ -140,11 +143,11 @@ protected void createPartitionReaders() throws IOException { partitionsChkPoints.put(id, null); } Date partitionTimestamp = getPartitionTimestamp(id, - partitionsChkPoints.get(id), allowedStartTime); + partitionsChkPoints.get(id), allowedStartTime); LOG.debug("Creating partition " + id); PartitionReaderStatsExposer collectorMetrics = new CollectorReaderStatsExposer(topicName, consumerName, - id.toString()); + id.toString(), consumerNumber); addStatsExposer(collectorMetrics); readers.put(id, new PartitionReader(id, partitionsChkPoints.get(id), conf, fs, @@ -157,20 +160,24 @@ protected void createPartitionReaders() throws IOException { } else { LOG.info("Creating partition reader for cluster"); PartitionId id = new PartitionId(clusterName, null); - if (partitionsChkPoints.get(id) == null) { - partitionsChkPoints.put(id, null); - } + Map listofPartitionCheckpoints = new + TreeMap(); + PartitionCheckpointList partitionCheckpointList = new + PartitionCheckpointList(listofPartitionCheckpoints); + ((CheckpointList)currentCheckpoint).preaprePartitionCheckPointList(id, + partitionCheckpointList); Date partitionTimestamp = getPartitionTimestamp(id, - partitionsChkPoints.get(id), allowedStartTime); + partitionCheckpointList, allowedStartTime); LOG.debug("Creating partition " + id); PartitionReaderStatsExposer clusterMetrics = new PartitionReaderStatsExposer(topicName, consumerName, - id.toString()); + id.toString(), consumerNumber); addStatsExposer(clusterMetrics); readers.put(id, new PartitionReader(id, - partitionsChkPoints.get(id), fs, buffer, streamDir, conf, + partitionCheckpointList, fs, buffer, streamDir, conf, TextInputFormat.class.getCanonicalName(), partitionTimestamp, - waitTimeForFileCreate, true, dataEncodingType, clusterMetrics)); + waitTimeForFileCreate, true, dataEncodingType, clusterMetrics, + partitionMinList)); } } } @@ -178,4 +185,16 @@ protected void createPartitionReaders() throws IOException { Path[] getRootDirs() { return rootDirs; } + + @Override + protected void createCheckpoint() { + // TODO Auto-generated method stub + Map partitionCheckpoint = new + HashMap(); + if (streamType.equals(StreamType.COLLECTOR)) { + currentCheckpoint = new Checkpoint(partitionCheckpoint); + } else { + currentCheckpoint = new CheckpointList(partitionMinList); + } + } } diff --git a/messaging-client-databus/src/main/java/com/inmobi/messaging/consumer/databus/MessageCheckpoint.java b/messaging-client-databus/src/main/java/com/inmobi/messaging/consumer/databus/MessageCheckpoint.java new file mode 100644 index 0000000..58942ac --- /dev/null +++ b/messaging-client-databus/src/main/java/com/inmobi/messaging/consumer/databus/MessageCheckpoint.java @@ -0,0 +1,6 @@ +package com.inmobi.messaging.consumer.databus; + +public interface MessageCheckpoint { + +} + diff --git a/messaging-client-databus/src/main/java/com/inmobi/messaging/consumer/databus/MessagingConsumerConfig.java b/messaging-client-databus/src/main/java/com/inmobi/messaging/consumer/databus/MessagingConsumerConfig.java index e949e5a..4c9f1eb 100644 --- a/messaging-client-databus/src/main/java/com/inmobi/messaging/consumer/databus/MessagingConsumerConfig.java +++ b/messaging-client-databus/src/main/java/com/inmobi/messaging/consumer/databus/MessagingConsumerConfig.java @@ -43,4 +43,12 @@ public interface MessagingConsumerConfig { public static final String hadoopConfigFileKey = "messaging.consumer.hadoop.conf"; + + /** + * The consumer id is used in case of groups. The number associated with + * consumer in the group, for eg. 2/5 + */ + public static final String consumerIdInGroupConfig = + "messaging.consumer.group.membership"; + public static final String DEFAULT_CONSUMER_ID = "1/1"; } diff --git a/messaging-client-databus/src/main/java/com/inmobi/messaging/consumer/databus/QueueEntry.java b/messaging-client-databus/src/main/java/com/inmobi/messaging/consumer/databus/QueueEntry.java index 7da82f5..0b7ae54 100644 --- a/messaging-client-databus/src/main/java/com/inmobi/messaging/consumer/databus/QueueEntry.java +++ b/messaging-client-databus/src/main/java/com/inmobi/messaging/consumer/databus/QueueEntry.java @@ -1,6 +1,5 @@ package com.inmobi.messaging.consumer.databus; -import com.inmobi.databus.partition.PartitionCheckpoint; import com.inmobi.databus.partition.PartitionId; import com.inmobi.messaging.Message; @@ -8,21 +7,21 @@ public class QueueEntry { private final Message message; private final PartitionId partitionId; - private final PartitionCheckpoint partitionChkpoint; + private final MessageCheckpoint msgCheckpoint; public QueueEntry(Message msg, PartitionId partitionId, - PartitionCheckpoint partitionChkpoint) { + MessageCheckpoint msgCheckpoint) { this.message = msg; this.partitionId = partitionId; - this.partitionChkpoint = partitionChkpoint; + this.msgCheckpoint = msgCheckpoint; } public PartitionId getPartitionId() { return partitionId; } - public PartitionCheckpoint getPartitionChkpoint() { - return partitionChkpoint; + public MessageCheckpoint getMessageChkpoint() { + return msgCheckpoint; } public Message getMessage() { diff --git a/messaging-client-databus/src/main/java/com/inmobi/messaging/consumer/hadoop/HadoopConsumer.java b/messaging-client-databus/src/main/java/com/inmobi/messaging/consumer/hadoop/HadoopConsumer.java index e58e9a8..28b228a 100644 --- a/messaging-client-databus/src/main/java/com/inmobi/messaging/consumer/hadoop/HadoopConsumer.java +++ b/messaging-client-databus/src/main/java/com/inmobi/messaging/consumer/hadoop/HadoopConsumer.java @@ -3,16 +3,19 @@ import java.io.IOException; import java.util.Date; import java.util.Map; +import java.util.TreeMap; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import com.inmobi.databus.partition.PartitionCheckpoint; +import com.inmobi.databus.partition.PartitionCheckpointList; import com.inmobi.databus.partition.PartitionId; import com.inmobi.databus.partition.PartitionReader; import com.inmobi.messaging.ClientConfig; import com.inmobi.messaging.consumer.databus.AbstractMessagingDatabusConsumer; +import com.inmobi.messaging.consumer.databus.CheckpointList; import com.inmobi.messaging.metrics.PartitionReaderStatsExposer; public class HadoopConsumer extends AbstractMessagingDatabusConsumer @@ -60,28 +63,32 @@ protected void createPartitionReaders() throws IOException { for (int i= 0; i < clusterNames.length; i++) { String clusterName = clusterNames[i]; LOG.debug("Creating partition reader for cluster:" + clusterName); - Map partitionsChkPoints = - currentCheckpoint.getPartitionsCheckpoint(); - + // create partition id PartitionId id = new PartitionId(clusterName, null); - if (partitionsChkPoints.get(id) == null) { - partitionsChkPoints.put(id, null); - } + Map listofPartitionCheckpoints = new + TreeMap(); + + PartitionCheckpointList partitionCheckpointList = new + PartitionCheckpointList(listofPartitionCheckpoints); + ((CheckpointList)currentCheckpoint).preaprePartitionCheckPointList(id, + partitionCheckpointList); // calculate the allowed start time long currentMillis = System.currentTimeMillis(); Date allowedStartTime = new Date(currentMillis - (retentionInHours * ONE_HOUR_IN_MILLIS)); Date partitionTimestamp = getPartitionTimestamp(id, - partitionsChkPoints.get(id), allowedStartTime); - PartitionReaderStatsExposer clusterMetrics = new - PartitionReaderStatsExposer(topicName, consumerName, id.toString()); + partitionCheckpointList, allowedStartTime); + PartitionReaderStatsExposer clusterMetrics = + new PartitionReaderStatsExposer(topicName, consumerName, id.toString(), + consumerNumber); addStatsExposer(clusterMetrics); PartitionReader reader = new PartitionReader(id, - partitionsChkPoints.get(id), fileSystems[i], buffer, rootDirs[i], + partitionCheckpointList, fileSystems[i], buffer, rootDirs[i], conf, inputFormatClassName, partitionTimestamp, - waitTimeForFileCreate, false, dataEncodingType, clusterMetrics); + waitTimeForFileCreate, false, dataEncodingType, clusterMetrics, + partitionMinList); LOG.debug("Created partition " + id); readers.put(id, reader); } @@ -94,4 +101,10 @@ Configuration getHadoopConf() { Path[] getRootDirs() { return rootDirs; } + + @Override + protected void createCheckpoint() { + currentCheckpoint = new CheckpointList(partitionMinList); + } + } diff --git a/messaging-client-databus/src/main/java/com/inmobi/messaging/consumer/util/CheckpointUtil.java b/messaging-client-databus/src/main/java/com/inmobi/messaging/consumer/util/CheckpointUtil.java new file mode 100644 index 0000000..3774d4c --- /dev/null +++ b/messaging-client-databus/src/main/java/com/inmobi/messaging/consumer/util/CheckpointUtil.java @@ -0,0 +1,236 @@ +package com.inmobi.messaging.consumer.util; + +import java.io.IOException; +import java.lang.reflect.Constructor; +import java.util.Calendar; +import java.util.Date; +import java.util.Map; +import java.util.Set; +import java.util.TreeMap; +import java.util.TreeSet; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.PathFilter; + +import com.inmobi.databus.CheckpointProvider; + +import com.inmobi.databus.files.FileMap; +import com.inmobi.databus.files.HadoopStreamFile; +import com.inmobi.databus.partition.PartitionCheckpoint; +import com.inmobi.databus.partition.PartitionCheckpointList; +import com.inmobi.databus.partition.PartitionId; +import com.inmobi.databus.readers.DatabusStreamWaitingReader; +import com.inmobi.messaging.ClientConfig; + +import com.inmobi.messaging.consumer.databus.Checkpoint; +import com.inmobi.messaging.consumer.databus.CheckpointList; +import com.inmobi.messaging.consumer.databus.DatabusConsumerConfig; +import com.inmobi.messaging.consumer.databus.StreamType; + + +public class CheckpointUtil implements DatabusConsumerConfig { + + private static final Log LOG = LogFactory.getLog(CheckpointUtil.class); + + public CheckpointUtil() { + + } + + public static void prepareCheckpointList(String superKey, + CheckpointProvider provider, + Set idList, Path streamDir, CheckpointList checkpointList) + throws IOException { + + Checkpoint oldCheckpoint = null; + byte[] chkpointData = provider.read(superKey); + if (chkpointData != null) { + oldCheckpoint = new Checkpoint(chkpointData); + } + + if (oldCheckpoint == null) { + LOG.info("Old checkpoint is not available nothing to prepare"); + return; + } + + Map partitionCheckpoints = + oldCheckpoint.getPartitionsCheckpoint(); + for (Map.Entry entry : + partitionCheckpoints.entrySet()) { + Path tmpPathFile = new Path(entry.getValue().getStreamFile().toString()). + getParent(); + //to get streamDir path form streamDirPath/YYYY/MM/DD/HH/MN) + for (int i = 0; i < 5; i++) { + tmpPathFile = tmpPathFile.getParent(); + } + if ((streamDir).compareTo(tmpPathFile) == 0) { + Map thisChkpoint = + new TreeMap(); + Calendar chkCal = Calendar.getInstance(); + Date checkpointDate = DatabusStreamWaitingReader.getDateFromStreamDir( + streamDir, + new Path(entry.getValue().getStreamFile().toString()).getParent()); + chkCal.setTime(checkpointDate); + int checkpointMin = chkCal.get(Calendar.MINUTE); + Calendar chkPrevHrCal = Calendar.getInstance(); + chkPrevHrCal.setTime(checkpointDate); + chkPrevHrCal.add(Calendar.MINUTE, 1); + chkPrevHrCal.add(Calendar.HOUR, -1); + while (chkPrevHrCal.before(chkCal)) { + if (chkPrevHrCal.get(Calendar.MINUTE) == 0) { + break; + } + Path minDir = DatabusStreamWaitingReader.getMinuteDirPath(streamDir, + chkPrevHrCal.getTime()); + FileStatus lastElement = getLast(minDir); + if (lastElement != null) { + thisChkpoint.put(chkPrevHrCal.get(Calendar.MINUTE), + new PartitionCheckpoint(new HadoopStreamFile( + lastElement.getPath().getParent(), + lastElement.getPath().getName(), + lastElement.getModificationTime()), + -1)); + } + chkPrevHrCal.add(Calendar.MINUTE, 1); + } + + Calendar chkHrCal = Calendar.getInstance(); + chkHrCal.setTime(checkpointDate); + chkHrCal.set(Calendar.MINUTE, 0); + while (chkHrCal.before(chkCal)) { + Path minDir = DatabusStreamWaitingReader.getMinuteDirPath(streamDir, + chkHrCal.getTime()); + FileStatus lastElement = getLast(minDir); + if (lastElement != null) { + thisChkpoint.put(chkHrCal.get(Calendar.MINUTE), + new PartitionCheckpoint(new HadoopStreamFile( + lastElement.getPath().getParent(), + lastElement.getPath().getName(), + lastElement.getModificationTime()), + -1)); + } + chkHrCal.add(Calendar.MINUTE, 1); + } + thisChkpoint.put(checkpointMin, entry.getValue()); + checkpointList.set(entry.getKey(), new PartitionCheckpointList( + thisChkpoint)); + } + } + } + + private static FileStatus getLast(final Path minDir) + throws IOException { + + FileMap fmap = new FileMap() { + @Override + protected TreeMap createFilesMap() { + return new TreeMap(); + } + + @Override + protected HadoopStreamFile getStreamFile(String fileName) { + throw new RuntimeException("Not implemented"); + } + + @Override + protected HadoopStreamFile getStreamFile(FileStatus file) { + return HadoopStreamFile.create(file); + } + + @Override + protected PathFilter createPathFilter() { + return new PathFilter() { + @Override + public boolean accept(Path path) { + if (path.getName().startsWith("_")) { + return false; + } + return true; + } + }; + } + + @Override + protected void buildList() throws IOException { + FileSystem fs = minDir.getFileSystem(new Configuration()); + doRecursiveListing(fs, minDir, pathFilter, this); + } + }; + + fmap.build(); + + return fmap.getLastFile(); + } + + private static void doRecursiveListing(FileSystem fs, Path dir, + PathFilter pathFilter, + FileMap fmap) throws IOException { + FileStatus[] fileStatuses = fs.listStatus(dir, pathFilter); + if (fileStatuses == null || fileStatuses.length == 0) { + LOG.debug("No files in directory:" + dir); + } else { + for (FileStatus file : fileStatuses) { + if (file.isDir()) { + doRecursiveListing(fs, file.getPath(), pathFilter, fmap); + } else { + fmap.addPath(file); + } + } + } + } + + protected static CheckpointProvider createCheckpointProvider( + String checkpointProviderClassName, String chkpointDir) { + CheckpointProvider chkProvider = null; + try { + Class clazz = Class.forName(checkpointProviderClassName); + Constructor constructor = clazz.getConstructor(String.class); + chkProvider = (CheckpointProvider) constructor.newInstance(new Object[] + {chkpointDir}); + } catch (Exception e) { + throw new IllegalArgumentException("Could not create checkpoint provider " + + checkpointProviderClassName, e); + } + return chkProvider; + } + + + public static void main(String [] args) throws Exception { + String confFile = args[0]; + ClientConfig config; + CheckpointProvider checkpointProvider; + Set idList = new TreeSet(); + config = ClientConfig.load(confFile); + String chkpointProviderClassName = config.getString( + chkProviderConfig, DEFAULT_CHK_PROVIDER); + String databusCheckpointDir = config.getString(checkpointDirConfig, + DEFAULT_CHECKPOINT_DIR); + checkpointProvider = createCheckpointProvider( + chkpointProviderClassName, databusCheckpointDir); + + for (int i = 0; i < 60; i++) { + idList.add(i); + } + String topicName = config.getString("topic.name", null); + String consumerName = config.getString("consumer.name", null); + String type = config.getString(databusStreamType, DEFAULT_STREAM_TYPE); + String [] databusRootDir = (config.getString(databusRootDirsConfig)).split(","); + StreamType streamType = StreamType.valueOf(type); + String superKey = consumerName + "_" + topicName; + CheckpointList checkpointList = new CheckpointList(idList); + for (String databusRootDirPath : databusRootDir) { + Path streamDir = DatabusUtil.getStreamDir(streamType, + new Path(databusRootDirPath), topicName); + CheckpointUtil.prepareCheckpointList(superKey, checkpointProvider, idList, + streamDir, checkpointList); + } + checkpointList.write(checkpointProvider, superKey); + checkpointList.read(checkpointProvider, superKey); + } +} + + diff --git a/messaging-client-databus/src/main/java/com/inmobi/messaging/metrics/CollectorReaderStatsExposer.java b/messaging-client-databus/src/main/java/com/inmobi/messaging/metrics/CollectorReaderStatsExposer.java index b98f7f1..10a9b2c 100644 --- a/messaging-client-databus/src/main/java/com/inmobi/messaging/metrics/CollectorReaderStatsExposer.java +++ b/messaging-client-databus/src/main/java/com/inmobi/messaging/metrics/CollectorReaderStatsExposer.java @@ -17,8 +17,8 @@ public class CollectorReaderStatsExposer extends PartitionReaderStatsExposer { private final AtomicLong numWaitTimeUnitsInSameFile = new AtomicLong(0); public CollectorReaderStatsExposer(String topicName, String consumerName, - String pid) { - super(topicName, consumerName, pid); + String pid, int consumerNumber) { + super(topicName, consumerName, pid, consumerNumber); } public void incrementSwitchesFromCollectorToLocal() { diff --git a/messaging-client-databus/src/main/java/com/inmobi/messaging/metrics/DatabusConsumerStatsExposer.java b/messaging-client-databus/src/main/java/com/inmobi/messaging/metrics/DatabusConsumerStatsExposer.java new file mode 100644 index 0000000..b7f4df2 --- /dev/null +++ b/messaging-client-databus/src/main/java/com/inmobi/messaging/metrics/DatabusConsumerStatsExposer.java @@ -0,0 +1,23 @@ +package com.inmobi.messaging.metrics; + +import java.util.Map; + +import com.inmobi.messaging.consumer.BaseMessageConsumerStatsExposer; + +public class DatabusConsumerStatsExposer extends + BaseMessageConsumerStatsExposer { + + public static String CONSUMER_NUMBER_CONTEXT = "consumerNumber"; + + Integer consumerNumber; + public DatabusConsumerStatsExposer(String topicName, String consumerName, + int consumerNumber) { + super(topicName, consumerName); + this.consumerNumber = consumerNumber; + } + + protected void addToContextsMap(Map contexts) { + super.addToContextsMap(contexts); + contexts.put(CONSUMER_NUMBER_CONTEXT, consumerNumber.toString()); + } +} diff --git a/messaging-client-databus/src/main/java/com/inmobi/messaging/metrics/PartitionReaderStatsExposer.java b/messaging-client-databus/src/main/java/com/inmobi/messaging/metrics/PartitionReaderStatsExposer.java index c54da36..88bd2f6 100644 --- a/messaging-client-databus/src/main/java/com/inmobi/messaging/metrics/PartitionReaderStatsExposer.java +++ b/messaging-client-databus/src/main/java/com/inmobi/messaging/metrics/PartitionReaderStatsExposer.java @@ -3,10 +3,8 @@ import java.util.Map; import java.util.concurrent.atomic.AtomicLong; -import com.inmobi.messaging.consumer.AbstractMessageConsumerStatsExposer; - public class PartitionReaderStatsExposer extends - AbstractMessageConsumerStatsExposer { + DatabusConsumerStatsExposer { public final static String MESSAGES_READ_FROM_SOURCE = "messagesReadFromSource"; public final static String MESSAGES_ADDED_TO_BUFFER = "messagesAddedToBuffer"; @@ -24,8 +22,8 @@ public class PartitionReaderStatsExposer extends private final String pid; public PartitionReaderStatsExposer(String topicName, String consumerName, - String pid) { - super(topicName, consumerName); + String pid, int consumerNumber) { + super(topicName, consumerName, consumerNumber); this.pid = pid; } diff --git a/messaging-client-databus/src/test/java/com/inmobi/databus/files/TestCollectorFile.java b/messaging-client-databus/src/test/java/com/inmobi/databus/files/TestCollectorFile.java index 86f7a37..7ffa8ef 100644 --- a/messaging-client-databus/src/test/java/com/inmobi/databus/files/TestCollectorFile.java +++ b/messaging-client-databus/src/test/java/com/inmobi/databus/files/TestCollectorFile.java @@ -11,7 +11,7 @@ public class TestCollectorFile { @Test public void testCollectorFile() throws ParseException { - Calendar now = Calendar.getInstance(); + Calendar now = Calendar.getInstance(); String stream1 = "a"; String stream2 = "b"; Date date = now.getTime(); @@ -69,6 +69,5 @@ public void testCollectorFile() throws ParseException { } Assert.assertNotNull(th); Assert.assertTrue(th instanceof IllegalArgumentException); - } } diff --git a/messaging-client-databus/src/test/java/com/inmobi/databus/partition/TestAbstractClusterReader.java b/messaging-client-databus/src/test/java/com/inmobi/databus/partition/TestAbstractClusterReader.java index bbdac11..a0d9fc6 100644 --- a/messaging-client-databus/src/test/java/com/inmobi/databus/partition/TestAbstractClusterReader.java +++ b/messaging-client-databus/src/test/java/com/inmobi/databus/partition/TestAbstractClusterReader.java @@ -2,13 +2,20 @@ import java.io.IOException; import java.util.Calendar; +import java.util.Date; +import java.util.Map; +import java.util.Set; +import java.util.TreeMap; +import java.util.TreeSet; import java.util.concurrent.LinkedBlockingQueue; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.mortbay.log.Log; import org.testng.Assert; +import com.inmobi.databus.files.StreamFile; import com.inmobi.databus.partition.PartitionCheckpoint; import com.inmobi.databus.partition.PartitionId; import com.inmobi.databus.partition.PartitionReader; @@ -26,6 +33,10 @@ public abstract class TestAbstractClusterReader { protected LinkedBlockingQueue buffer = new LinkedBlockingQueue(1000); protected PartitionReader preader; + public Set partitionMinList; + PartitionCheckpointList partitionCheckpointList; + Map pchkPoints; + int consumerNumber; protected String[] files = new String[] {TestUtil.files[1], TestUtil.files[3], TestUtil.files[5]}; @@ -46,15 +57,20 @@ public void cleanup() throws IOException { abstract boolean isDatabusData(); public void testInitialize() throws Exception { - PartitionReaderStatsExposer prMetrics = new PartitionReaderStatsExposer( - testStream, "c1", partitionId.toString()); + initializeMinList(); + Map chkpoints = new TreeMap(); + partitionCheckpointList = new PartitionCheckpointList(chkpoints); + + PartitionReaderStatsExposer prMetrics = new PartitionReaderStatsExposer( + testStream, "c1", partitionId.toString(), consumerNumber); Calendar cal = Calendar.getInstance(); cal.setTime(DatabusStreamWaitingReader.getDateFromStreamDir(streamDir, databusFiles[0].getParent())); // Read from start preader = new PartitionReader(partitionId, null, fs, buffer, streamDir, conf, inputFormatClass, cal.getTime(), 1000, - isDatabusData(), dataEncoding, prMetrics); + isDatabusData(), dataEncoding, prMetrics, partitionMinList); Assert.assertEquals(preader.getReader().getClass().getName(), ClusterReader.class.getName()); preader.init(); @@ -62,21 +78,24 @@ public void testInitialize() throws Exception { databusFiles[0].toString()); // Read from checkpoint with local stream file name - preader = new PartitionReader(partitionId, new PartitionCheckpoint( - DatabusStreamWaitingReader.getHadoopStreamFile( - fs.getFileStatus(databusFiles[1])), 20), fs, buffer, + prepareCheckpoint(DatabusStreamWaitingReader.getHadoopStreamFile( + fs.getFileStatus(databusFiles[1])), 20, databusFiles[1], + partitionCheckpointList); + preader = new PartitionReader(partitionId, partitionCheckpointList, fs, buffer, streamDir, conf, inputFormatClass, null, 1000, - isDatabusData(), dataEncoding, prMetrics); + isDatabusData(), dataEncoding, prMetrics, partitionMinList); preader.init(); Assert.assertEquals(preader.getCurrentFile().toString(), databusFiles[1].toString()); // Read from checkpoint with local stream file name which does not exist // and is before the stream - preader = new PartitionReader(partitionId, new PartitionCheckpoint( - HadoopUtil.getOlderFile(streamDir, fs, databusFiles[0]), 20), + prepareCheckpoint(HadoopUtil.getOlderFile(streamDir, fs, databusFiles[0]), + 20, databusFiles[1], partitionCheckpointList); + + preader = new PartitionReader(partitionId, partitionCheckpointList, fs, buffer, streamDir, conf, inputFormatClass, null, 1000, - isDatabusData(), dataEncoding, prMetrics); + isDatabusData(), dataEncoding, prMetrics, partitionMinList); preader.init(); Assert.assertEquals(preader.getCurrentFile().toString(), databusFiles[0].toString()); @@ -87,17 +106,18 @@ public void testInitialize() throws Exception { preader = new PartitionReader(partitionId, null, fs, buffer, streamDir, conf, inputFormatClass, cal.getTime(), 1000, - isDatabusData(), dataEncoding, prMetrics); + isDatabusData(), dataEncoding, prMetrics, partitionMinList); preader.init(); Assert.assertEquals(preader.getCurrentFile().toString(), databusFiles[1].toString()); //Read from startTime in local stream directory, with checkpoint - preader = new PartitionReader(partitionId, new PartitionCheckpoint( - DatabusStreamWaitingReader.getHadoopStreamFile( - fs.getFileStatus(databusFiles[1])), 20),fs, buffer, + prepareCheckpoint(DatabusStreamWaitingReader.getHadoopStreamFile( + fs.getFileStatus(databusFiles[1])), 20, databusFiles[1], + partitionCheckpointList); + preader = new PartitionReader(partitionId, partitionCheckpointList,fs, buffer, streamDir, conf, inputFormatClass, cal.getTime(), 1000, - isDatabusData(), dataEncoding, prMetrics); + isDatabusData(), dataEncoding, prMetrics, partitionMinList); preader.init(); Assert.assertEquals(preader.getCurrentFile().toString(), databusFiles[1].toString()); @@ -109,18 +129,19 @@ public void testInitialize() throws Exception { cal.add(Calendar.MINUTE, 1); preader = new PartitionReader(partitionId, null, fs, buffer, streamDir, conf, inputFormatClass, cal.getTime(), 1000, - isDatabusData(), dataEncoding, prMetrics); + isDatabusData(), dataEncoding, prMetrics, partitionMinList); preader.init(); Assert.assertEquals(preader.getCurrentFile().toString(), databusFiles[1].toString()); //Read from startTime in local stream directory, with no timestamp file, //with checkpoint - preader = new PartitionReader(partitionId, new PartitionCheckpoint( - DatabusStreamWaitingReader.getHadoopStreamFile( - fs.getFileStatus(databusFiles[1])), 20), fs, buffer, + prepareCheckpoint(DatabusStreamWaitingReader.getHadoopStreamFile( + fs.getFileStatus(databusFiles[1])), 20, databusFiles[1], + partitionCheckpointList); + preader = new PartitionReader(partitionId,partitionCheckpointList, fs, buffer, streamDir, conf, inputFormatClass, cal.getTime(), 1000, - isDatabusData(), dataEncoding, prMetrics); + isDatabusData(), dataEncoding, prMetrics, partitionMinList); preader.init(); Assert.assertEquals(preader.getCurrentFile().toString(), databusFiles[1].toString()); @@ -131,17 +152,18 @@ public void testInitialize() throws Exception { cal.add(Calendar.MINUTE, -2); preader = new PartitionReader(partitionId, null, fs, buffer, streamDir, conf, inputFormatClass, cal.getTime(), 1000, - isDatabusData(), dataEncoding, prMetrics); + isDatabusData(), dataEncoding, prMetrics, partitionMinList); preader.init(); Assert.assertEquals(preader.getCurrentFile().toString(), databusFiles[0].toString()); //Read from startTime beyond the stream, with checkpoint - preader = new PartitionReader(partitionId, new PartitionCheckpoint( - DatabusStreamWaitingReader.getHadoopStreamFile( - fs.getFileStatus(databusFiles[1])), 20), fs, buffer, + prepareCheckpoint(DatabusStreamWaitingReader.getHadoopStreamFile( + fs.getFileStatus(databusFiles[1])), 20, databusFiles[1], + partitionCheckpointList); + preader = new PartitionReader(partitionId, partitionCheckpointList, fs, buffer, streamDir, conf, inputFormatClass, cal.getTime(), 1000, - isDatabusData(), dataEncoding, prMetrics); + isDatabusData(), dataEncoding, prMetrics, partitionMinList); preader.init(); Assert.assertEquals(preader.getCurrentFile().toString(), databusFiles[0].toString()); @@ -153,7 +175,7 @@ public void testInitialize() throws Exception { preader = new PartitionReader(partitionId, null, fs, buffer, streamDir, conf, inputFormatClass, cal.getTime(), 1000, - isDatabusData(), dataEncoding, prMetrics, true); + isDatabusData(), dataEncoding, prMetrics, true, partitionMinList); preader.init(); Assert.assertNotNull(preader.getReader()); Assert.assertEquals(preader.getReader().getClass().getName(), @@ -164,11 +186,12 @@ public void testInitialize() throws Exception { Assert.assertNull(preader.getCurrentFile()); //Read from startTime after the stream, with checkpoint - preader = new PartitionReader(partitionId, new PartitionCheckpoint( - DatabusStreamWaitingReader.getHadoopStreamFile( - fs.getFileStatus(databusFiles[1])), 20), fs, buffer, + prepareCheckpoint(DatabusStreamWaitingReader.getHadoopStreamFile( + fs.getFileStatus(databusFiles[1])), 20, databusFiles[1], + partitionCheckpointList); + preader = new PartitionReader(partitionId, partitionCheckpointList, fs, buffer, streamDir, conf, inputFormatClass, cal.getTime(), 1000, - isDatabusData(), dataEncoding, prMetrics, true); + isDatabusData(), dataEncoding, prMetrics, true, partitionMinList); preader.init(); Assert.assertNotNull(preader.getReader()); Assert.assertEquals(preader.getReader().getClass().getName(), @@ -180,14 +203,17 @@ public void testInitialize() throws Exception { } public void testReadFromStart() throws Exception { + initializeMinList(); + Map chkpoints = new TreeMap(); + partitionCheckpointList = new PartitionCheckpointList(chkpoints); PartitionReaderStatsExposer prMetrics = new PartitionReaderStatsExposer( - testStream, "c1", partitionId.toString()); - preader = new PartitionReader(partitionId, null, fs, buffer, streamDir, - conf, inputFormatClass, + testStream, "c1", partitionId.toString(), consumerNumber); + preader = new PartitionReader(partitionId, partitionCheckpointList, fs, + buffer, streamDir, conf, inputFormatClass, DatabusStreamWaitingReader.getDateFromStreamDir(streamDir, databusFiles[0].getParent()), - 1000, - isDatabusData(), dataEncoding, prMetrics, true); + 1000, isDatabusData(), dataEncoding, prMetrics, true, partitionMinList); preader.init(); Assert.assertTrue(buffer.isEmpty()); Assert.assertEquals(preader.getReader().getClass().getName(), @@ -220,13 +246,19 @@ public void testReadFromStart() throws Exception { } public void testReadFromCheckpoint() throws Exception { + initializeMinList(); + Map chkpoints = new TreeMap(); + partitionCheckpointList = new PartitionCheckpointList(chkpoints); PartitionReaderStatsExposer prMetrics = new PartitionReaderStatsExposer( - testStream, "c1", partitionId.toString()); - preader = new PartitionReader(partitionId, new PartitionCheckpoint( - DatabusStreamWaitingReader.getHadoopStreamFile(fs.getFileStatus( - databusFiles[1])), 20), fs, buffer, + testStream, "c1", partitionId.toString(), consumerNumber); + prepareCheckpoint(DatabusStreamWaitingReader.getHadoopStreamFile( + fs.getFileStatus(databusFiles[1])), 20, databusFiles[1], + partitionCheckpointList); + + preader = new PartitionReader(partitionId, partitionCheckpointList, fs, buffer, streamDir, conf, inputFormatClass, null, 1000, - isDatabusData(), dataEncoding, prMetrics, true); + isDatabusData(), dataEncoding, prMetrics, true, partitionMinList); preader.init(); Assert.assertTrue(buffer.isEmpty()); Assert.assertEquals(preader.getReader().getClass().getName(), @@ -256,13 +288,18 @@ public void testReadFromCheckpoint() throws Exception { } public void testReadFromCheckpointWhichDoesNotExist() throws Exception { + initializeMinList(); + Map chkpoints = new TreeMap(); + partitionCheckpointList = new PartitionCheckpointList(chkpoints); PartitionReaderStatsExposer prMetrics = new PartitionReaderStatsExposer( - testStream, "c1", partitionId.toString()); - preader = new PartitionReader(partitionId, new PartitionCheckpoint( - HadoopUtil.getOlderFile(streamDir, fs, databusFiles[0]), - 20), fs, buffer, + testStream, "c1", partitionId.toString(), consumerNumber); + prepareCheckpoint(HadoopUtil.getOlderFile(streamDir, fs, databusFiles[0]), + 20, databusFiles[0], partitionCheckpointList); + + preader = new PartitionReader(partitionId, partitionCheckpointList, fs, buffer, streamDir, conf, inputFormatClass, null, 1000, - isDatabusData(), dataEncoding, prMetrics, true); + isDatabusData(), dataEncoding, prMetrics, true, partitionMinList); preader.init(); Assert.assertTrue(buffer.isEmpty()); Assert.assertEquals(preader.getReader().getClass().getName(), @@ -295,16 +332,22 @@ public void testReadFromCheckpointWhichDoesNotExist() throws Exception { } public void testReadFromStartTime() throws Exception { + initializeMinList(); + Map chkpoints = new TreeMap(); + partitionCheckpointList = new PartitionCheckpointList(chkpoints); PartitionReaderStatsExposer prMetrics = new PartitionReaderStatsExposer( - testStream, "c1", partitionId.toString()); - preader = new PartitionReader(partitionId, new PartitionCheckpoint( - DatabusStreamWaitingReader.getHadoopStreamFile( - fs.getFileStatus(databusFiles[1])), 20), fs, buffer, - streamDir, conf, inputFormatClass, + testStream, "c1", partitionId.toString(), consumerNumber); + prepareCheckpoint(DatabusStreamWaitingReader.getHadoopStreamFile( + fs.getFileStatus(databusFiles[1])), 20, databusFiles[1], + partitionCheckpointList); + + preader = new PartitionReader(partitionId, partitionCheckpointList, fs, + buffer, streamDir, conf, inputFormatClass, DatabusStreamWaitingReader.getDateFromStreamDir(streamDir, databusFiles[1].getParent()), 1000, - isDatabusData(), dataEncoding, prMetrics, true); + isDatabusData(), dataEncoding, prMetrics, true, partitionMinList); preader.init(); Assert.assertTrue(buffer.isEmpty()); Assert.assertEquals(preader.getReader().getClass().getName(), @@ -334,17 +377,23 @@ public void testReadFromStartTime() throws Exception { } public void testReadFromStartTimeWithinStream() throws Exception { + initializeMinList(); + Map chkpoints = new TreeMap(); + partitionCheckpointList = new PartitionCheckpointList(chkpoints); Calendar cal = Calendar.getInstance(); cal.setTime(DatabusStreamWaitingReader.getDateFromStreamDir(streamDir, databusFiles[0].getParent())); cal.add(Calendar.MINUTE, 1); PartitionReaderStatsExposer prMetrics = new PartitionReaderStatsExposer( - testStream, "c1", partitionId.toString()); - preader = new PartitionReader(partitionId, new PartitionCheckpoint( - DatabusStreamWaitingReader.getHadoopStreamFile( - fs.getFileStatus(databusFiles[1])), 20), fs, buffer, + testStream, "c1", partitionId.toString(), consumerNumber); + prepareCheckpoint(DatabusStreamWaitingReader.getHadoopStreamFile( + fs.getFileStatus(databusFiles[1])), 20, databusFiles[1], + partitionCheckpointList); + + preader = new PartitionReader(partitionId, partitionCheckpointList, fs, buffer, streamDir, conf, inputFormatClass, cal.getTime(), 1000, - isDatabusData(), dataEncoding, prMetrics, true); + isDatabusData(), dataEncoding, prMetrics, true, partitionMinList); preader.init(); Assert.assertTrue(buffer.isEmpty()); Assert.assertEquals(preader.getReader().getClass().getName(), @@ -370,21 +419,27 @@ public void testReadFromStartTimeWithinStream() throws Exception { Assert.assertEquals(prMetrics.getMessagesReadFromSource(), 200); Assert.assertEquals(prMetrics.getMessagesAddedToBuffer(), 200); Assert.assertEquals(prMetrics.getWaitTimeUnitsNewFile(), 0); - Assert.assertTrue(prMetrics.getCumulativeNanosForFetchMessage() > 0); + Assert.assertTrue(prMetrics.getCumulativeNanosForFetchMessage() > 0); } public void testReadFromStartTimeBeforeStream() throws Exception { + initializeMinList(); + Map chkpoints = new TreeMap(); + partitionCheckpointList = new PartitionCheckpointList(chkpoints); Calendar cal = Calendar.getInstance(); cal.setTime(DatabusStreamWaitingReader.getDateFromStreamDir(streamDir, databusFiles[0].getParent())); cal.add(Calendar.MINUTE, -1); PartitionReaderStatsExposer prMetrics = new PartitionReaderStatsExposer( - testStream, "c1", partitionId.toString()); - preader = new PartitionReader(partitionId, new PartitionCheckpoint( - DatabusStreamWaitingReader.getHadoopStreamFile( - fs.getFileStatus(databusFiles[1])), 20), fs, buffer, + testStream, "c1", partitionId.toString(), consumerNumber); + prepareCheckpoint(DatabusStreamWaitingReader.getHadoopStreamFile( + fs.getFileStatus(databusFiles[1])), 20, databusFiles[1], + partitionCheckpointList); + + preader = new PartitionReader(partitionId, partitionCheckpointList, fs, buffer, streamDir, conf, inputFormatClass, cal.getTime(), 1000, - isDatabusData(), dataEncoding, prMetrics, true); + isDatabusData(), dataEncoding, prMetrics, true, partitionMinList); preader.init(); Assert.assertTrue(buffer.isEmpty()); Assert.assertEquals(preader.getReader().getClass().getName(), @@ -397,7 +452,7 @@ public void testReadFromStartTimeBeforeStream() throws Exception { fs.getFileStatus(databusFiles[0])), 1, 0, 100, partitionId, buffer, dataEncoding.equals(DataEncodingType.BASE64)); TestUtil.assertBuffer(DatabusStreamWaitingReader.getHadoopStreamFile( - fs.getFileStatus(databusFiles[1])), 2, 0, 100, partitionId, + fs.getFileStatus(databusFiles[1])), 2, 20, 80, partitionId, buffer, dataEncoding.equals(DataEncodingType.BASE64)); TestUtil.assertBuffer(DatabusStreamWaitingReader.getHadoopStreamFile( fs.getFileStatus(databusFiles[2])), 3, 0, 100, partitionId, @@ -410,24 +465,29 @@ public void testReadFromStartTimeBeforeStream() throws Exception { .getReader().getClass().getName(), DatabusStreamWaitingReader.class.getName()); Assert.assertEquals(prMetrics.getHandledExceptions(), 0); - Assert.assertEquals(prMetrics.getMessagesReadFromSource(), 300); - Assert.assertEquals(prMetrics.getMessagesAddedToBuffer(), 300); + Assert.assertEquals(prMetrics.getMessagesReadFromSource(), 280); + Assert.assertEquals(prMetrics.getMessagesAddedToBuffer(), 280); Assert.assertEquals(prMetrics.getWaitTimeUnitsNewFile(), 0); - Assert.assertTrue(prMetrics.getCumulativeNanosForFetchMessage() > 0); + Assert.assertTrue(prMetrics.getCumulativeNanosForFetchMessage() > 0); } public void testReadFromStartTimeAfterStream() throws Exception { + initializeMinList(); + Map chkpoints = new TreeMap(); + partitionCheckpointList = new PartitionCheckpointList(chkpoints); Calendar cal = Calendar.getInstance(); cal.setTime(DatabusStreamWaitingReader.getDateFromStreamDir(streamDir, databusFiles[2].getParent())); cal.add(Calendar.MINUTE, 2); PartitionReaderStatsExposer prMetrics = new PartitionReaderStatsExposer( - testStream, "c1", partitionId.toString()); - preader = new PartitionReader(partitionId, new PartitionCheckpoint( - DatabusStreamWaitingReader.getHadoopStreamFile( - fs.getFileStatus(databusFiles[1])), 20), fs, buffer, + testStream, "c1", partitionId.toString(), consumerNumber); + prepareCheckpoint(DatabusStreamWaitingReader.getHadoopStreamFile( + fs.getFileStatus(databusFiles[1])), 20, databusFiles[1], + partitionCheckpointList); + preader = new PartitionReader(partitionId, partitionCheckpointList, fs, buffer, streamDir, conf, inputFormatClass, cal.getTime(), 1000, - isDatabusData(), dataEncoding, prMetrics, true); + isDatabusData(), dataEncoding, prMetrics, true, partitionMinList); preader.init(); Assert.assertTrue(buffer.isEmpty()); Assert.assertNotNull(preader.getReader()); @@ -441,4 +501,179 @@ public void testReadFromStartTimeAfterStream() throws Exception { Assert.assertEquals(prMetrics.getMessagesAddedToBuffer(), 0); Assert.assertEquals(prMetrics.getCumulativeNanosForFetchMessage(), 0); } + + public void testReadFromCheckpointWithSingleMinute() throws Exception { + partitionMinList = new TreeSet(); + Map chkpoints = new + TreeMap(); + partitionCheckpointList = new PartitionCheckpointList(chkpoints); + + for (int i =0; i < 1; i++) { + Date date = DatabusStreamWaitingReader.getDateFromStreamDir(streamDir, + databusFiles[i].getParent()) ; + Log.info("date is " + date); + partitionMinList.add(date.getMinutes()); + partitionCheckpointList.set(date.getMinutes(), new PartitionCheckpoint( + DatabusStreamWaitingReader.getHadoopStreamFile(fs.getFileStatus( + databusFiles[i])), 20)); + } + + PartitionReaderStatsExposer prMetrics = new PartitionReaderStatsExposer( + testStream, "c1", partitionId.toString(), consumerNumber); + Calendar cal = Calendar.getInstance(); + cal.setTime(DatabusStreamWaitingReader.getDateFromStreamDir(streamDir, + databusFiles[0].getParent())); + + preader = new PartitionReader(partitionId, partitionCheckpointList, fs, buffer, + streamDir, conf, inputFormatClass, null, 1000, + isDatabusData(), dataEncoding, prMetrics, true, partitionMinList); + preader.init(); + Assert.assertEquals(preader.getCurrentFile().toString(), + databusFiles[0].toString()); + + preader.execute(); + TestUtil.assertBuffer(DatabusStreamWaitingReader.getHadoopStreamFile( + fs.getFileStatus(databusFiles[0])), 1, 20, 80, partitionId, + buffer, dataEncoding.equals(DataEncodingType.BASE64)); + + Assert.assertTrue(buffer.isEmpty()); + Assert.assertNotNull(preader.getReader()); + Assert.assertEquals(preader.getReader().getClass().getName(), + ClusterReader.class.getName()); + Assert.assertEquals(((ClusterReader)preader.getReader()) + .getReader().getClass().getName(), + DatabusStreamWaitingReader.class.getName()); + Assert.assertEquals(prMetrics.getHandledExceptions(), 0); + Assert.assertEquals(prMetrics.getMessagesReadFromSource(), 80); + Assert.assertEquals(prMetrics.getMessagesAddedToBuffer(), 80); + Assert.assertEquals(prMetrics.getWaitTimeUnitsNewFile(), 0); + Assert.assertTrue(prMetrics.getCumulativeNanosForFetchMessage() > 0); + } + + public void testReadFromCheckpointMultipleMinutes() throws Exception { + partitionMinList = new TreeSet(); + Map chkpoints = new + TreeMap(); + partitionCheckpointList = new PartitionCheckpointList(chkpoints); + + for (int i =0; i < 3; i++) { + Date date = DatabusStreamWaitingReader.getDateFromStreamDir(streamDir, + databusFiles[i].getParent()) ; + Log.info("date is " + date); + partitionMinList.add(date.getMinutes()); + partitionCheckpointList.set(date.getMinutes(), new PartitionCheckpoint( + DatabusStreamWaitingReader.getHadoopStreamFile(fs.getFileStatus( + databusFiles[i])), 20)); + } + + PartitionReaderStatsExposer prMetrics = new PartitionReaderStatsExposer( + testStream, "c1", partitionId.toString(), consumerNumber); + Calendar cal = Calendar.getInstance(); + cal.setTime(DatabusStreamWaitingReader.getDateFromStreamDir(streamDir, + databusFiles[0].getParent())); + + preader = new PartitionReader(partitionId, partitionCheckpointList, fs, buffer, + streamDir, conf, inputFormatClass, null, 1000, + isDatabusData(), dataEncoding, prMetrics,true, partitionMinList); + preader.init(); + Assert.assertEquals(preader.getCurrentFile().toString(), + databusFiles[0].toString()); + + preader.execute(); + TestUtil.assertBuffer(DatabusStreamWaitingReader.getHadoopStreamFile( + fs.getFileStatus(databusFiles[0])), 1, 20, 80, partitionId, + buffer, dataEncoding.equals(DataEncodingType.BASE64)); + + TestUtil.assertBuffer(DatabusStreamWaitingReader.getHadoopStreamFile( + fs.getFileStatus(databusFiles[1])), 2, 20, 80, partitionId, + buffer, dataEncoding.equals(DataEncodingType.BASE64)); + TestUtil.assertBuffer(DatabusStreamWaitingReader.getHadoopStreamFile( + fs.getFileStatus(databusFiles[2])), 3, 20, 80, partitionId, + buffer, dataEncoding.equals(DataEncodingType.BASE64)); + Assert.assertTrue(buffer.isEmpty()); + Assert.assertNotNull(preader.getReader()); + Assert.assertEquals(preader.getReader().getClass().getName(), + ClusterReader.class.getName()); + Assert.assertEquals(((ClusterReader)preader.getReader()) + .getReader().getClass().getName(), + DatabusStreamWaitingReader.class.getName()); + Assert.assertEquals(prMetrics.getHandledExceptions(), 0); + Assert.assertEquals(prMetrics.getMessagesReadFromSource(), 240); + Assert.assertEquals(prMetrics.getMessagesAddedToBuffer(), 240); + Assert.assertEquals(prMetrics.getWaitTimeUnitsNewFile(), 0); + Assert.assertTrue(prMetrics.getCumulativeNanosForFetchMessage() > 0); + } + + public void testReadFromCheckpointSomeMinutes() throws Exception { + partitionMinList = new TreeSet(); + Map chkpoints = new + TreeMap(); + partitionCheckpointList = new PartitionCheckpointList(chkpoints); + for (int i =0; i < 3; i++) { + Date date = DatabusStreamWaitingReader.getDateFromStreamDir(streamDir, + databusFiles[i].getParent()) ; + Log.info("date is " + date); + partitionMinList.add(date.getMinutes()); + if (i != 1) { + partitionCheckpointList.set(date.getMinutes(), new PartitionCheckpoint( + DatabusStreamWaitingReader.getHadoopStreamFile(fs.getFileStatus( + databusFiles[i])), 20)); + } else { + partitionCheckpointList.set(date.getMinutes(), new PartitionCheckpoint( + DatabusStreamWaitingReader.getHadoopStreamFile(fs.getFileStatus( + databusFiles[i])), 00)); + } + } + + PartitionReaderStatsExposer prMetrics = new PartitionReaderStatsExposer( + testStream, "c1", partitionId.toString(), consumerNumber); + Calendar cal = Calendar.getInstance(); + cal.setTime(DatabusStreamWaitingReader.getDateFromStreamDir(streamDir, + databusFiles[0].getParent())); + + preader = new PartitionReader(partitionId, partitionCheckpointList, fs, buffer, + streamDir, conf, inputFormatClass, null, 1000, + isDatabusData(), dataEncoding, prMetrics, true, partitionMinList); + preader.init(); + Assert.assertEquals(preader.getCurrentFile().toString(), + databusFiles[0].toString()); + + preader.execute(); + TestUtil.assertBuffer(DatabusStreamWaitingReader.getHadoopStreamFile( + fs.getFileStatus(databusFiles[1])), 1, 20, 80, partitionId, + buffer, dataEncoding.equals(DataEncodingType.BASE64)); + TestUtil.assertBuffer(DatabusStreamWaitingReader.getHadoopStreamFile( + fs.getFileStatus(databusFiles[1])), 2, 0, 100, partitionId, + buffer, dataEncoding.equals(DataEncodingType.BASE64)); + TestUtil.assertBuffer(DatabusStreamWaitingReader.getHadoopStreamFile( + fs.getFileStatus(databusFiles[2])), 3, 20, 80, partitionId, + buffer, dataEncoding.equals(DataEncodingType.BASE64)); + Assert.assertTrue(buffer.isEmpty()); + Assert.assertNotNull(preader.getReader()); + Assert.assertEquals(preader.getReader().getClass().getName(), + ClusterReader.class.getName()); + Assert.assertEquals(((ClusterReader)preader.getReader()) + .getReader().getClass().getName(), + DatabusStreamWaitingReader.class.getName()); + Assert.assertEquals(prMetrics.getHandledExceptions(), 0); + Assert.assertEquals(prMetrics.getMessagesReadFromSource(), 260); + Assert.assertEquals(prMetrics.getMessagesAddedToBuffer(), 260); + Assert.assertEquals(prMetrics.getWaitTimeUnitsNewFile(), 0); + Assert.assertTrue(prMetrics.getCumulativeNanosForFetchMessage() > 0); + } + + public void prepareCheckpoint(StreamFile streamFile, int lineNum, + Path databusFile, PartitionCheckpointList partitionCheckpointList) { + Date date = DatabusStreamWaitingReader.getDateFromStreamDir(streamDir, + databusFile.getParent()); + partitionCheckpointList.set(date.getMinutes(), new PartitionCheckpoint( + streamFile, lineNum)); + } + + public void initializeMinList() { + partitionMinList = new TreeSet(); + for (int i =0; i < 60; i++) { + partitionMinList.add(i); + } + } } diff --git a/messaging-client-databus/src/test/java/com/inmobi/databus/partition/TestAbstractWaitingClusterReader.java b/messaging-client-databus/src/test/java/com/inmobi/databus/partition/TestAbstractWaitingClusterReader.java index 723c9de..6733c12 100644 --- a/messaging-client-databus/src/test/java/com/inmobi/databus/partition/TestAbstractWaitingClusterReader.java +++ b/messaging-client-databus/src/test/java/com/inmobi/databus/partition/TestAbstractWaitingClusterReader.java @@ -1,6 +1,7 @@ package com.inmobi.databus.partition; import java.io.IOException; +import java.util.Set; import java.util.concurrent.LinkedBlockingQueue; import org.apache.hadoop.conf.Configuration; @@ -26,7 +27,10 @@ public abstract class TestAbstractWaitingClusterReader { new LinkedBlockingQueue(150); protected Cluster cluster; protected PartitionReader preader; + Set partitionMinList; + PartitionCheckpointList partitionCheckpointlist; + protected String[] files = new String[] {TestUtil.files[1], TestUtil.files[3], TestUtil.files[5]}; protected String[] newFiles = new String[] {TestUtil.files[6], TestUtil.files[7], @@ -39,6 +43,7 @@ public abstract class TestAbstractWaitingClusterReader { DataEncodingType dataEncoding; Path streamDir; Configuration conf; + int consumerNumber; abstract void setupFiles(String[] files, Path[] newDatabusFiles) throws Exception; @@ -50,12 +55,12 @@ public void cleanup() throws IOException { public void testReadFromStart() throws Exception { PartitionReaderStatsExposer prMetrics = new PartitionReaderStatsExposer( - testStream, "c1", partitionId.toString()); - preader = new PartitionReader(partitionId, null, fs, buffer, - streamDir, conf, inputFormatClass, + testStream, "c1", partitionId.toString(), consumerNumber); + preader = new PartitionReader(partitionId, partitionCheckpointlist, fs, + buffer, streamDir, conf, inputFormatClass, DatabusStreamWaitingReader.getDateFromStreamDir(streamDir, databusFiles[0]), - 1000, isDatabusData(), dataEncoding, prMetrics, false); + 1000, isDatabusData(), dataEncoding, prMetrics, false, partitionMinList); preader.init(); Assert.assertTrue(buffer.isEmpty()); diff --git a/messaging-client-databus/src/test/java/com/inmobi/databus/partition/TestClusterReaderEmptyStream.java b/messaging-client-databus/src/test/java/com/inmobi/databus/partition/TestClusterReaderEmptyStream.java index bdabb1a..56291b1 100644 --- a/messaging-client-databus/src/test/java/com/inmobi/databus/partition/TestClusterReaderEmptyStream.java +++ b/messaging-client-databus/src/test/java/com/inmobi/databus/partition/TestClusterReaderEmptyStream.java @@ -1,6 +1,11 @@ package com.inmobi.databus.partition; import java.io.IOException; +import java.util.Date; +import java.util.Map; +import java.util.Set; +import java.util.TreeMap; +import java.util.TreeSet; import java.util.concurrent.LinkedBlockingQueue; import org.apache.hadoop.conf.Configuration; @@ -13,6 +18,7 @@ import org.testng.annotations.Test; import com.inmobi.databus.files.HadoopStreamFile; +import com.inmobi.databus.files.StreamFile; import com.inmobi.databus.partition.PartitionCheckpoint; import com.inmobi.databus.partition.PartitionId; import com.inmobi.databus.partition.PartitionReader; @@ -33,20 +39,31 @@ public class TestClusterReaderEmptyStream { private PartitionReader preader; private static final String clusterName = "testCluster"; private PartitionId clusterId = new PartitionId(clusterName, null); - + Set partitionMinList; + PartitionCheckpointList partitionCheckpointList; + Map chkPoints; + FileSystem fs; Path streamDir; Configuration conf = new Configuration(); String inputFormatClass; + int consumerNumber; @BeforeTest public void setup() throws Exception { // setup cluster + consumerNumber = 1; fs = FileSystem.getLocal(conf); streamDir = new Path("/tmp/test/hadoop/" + this.getClass().getSimpleName(), testStream).makeQualified(fs); HadoopUtil.setupHadoopCluster(conf, null, null, null, streamDir); inputFormatClass = SequenceFileInputFormat.class.getName(); + partitionMinList = new TreeSet(); + for (int i = 0; i < 60; i++) { + partitionMinList.add(i); + } + chkPoints = new TreeMap(); + partitionCheckpointList = new PartitionCheckpointList(chkPoints); } @AfterTest @@ -57,13 +74,13 @@ public void cleanup() throws IOException { @Test public void testInitialize() throws Exception { PartitionReaderStatsExposer prMetrics = new PartitionReaderStatsExposer( - testStream, "c1", clusterId.toString()); + testStream, "c1", clusterId.toString(), consumerNumber); // Read from start time preader = new PartitionReader(clusterId, null, fs, buffer, streamDir, conf, inputFormatClass, CollectorStreamReader.getDateFromCollectorFile(TestUtil.files[0]), 1000, - false, DataEncodingType.BASE64, prMetrics, true); + false, DataEncodingType.BASE64, prMetrics, true, partitionMinList); preader.init(); Assert.assertNotNull(preader.getReader()); Assert.assertEquals(preader.getReader().getClass().getName(), @@ -73,12 +90,13 @@ public void testInitialize() throws Exception { DatabusStreamWaitingReader.class.getName()); //Read from checkpoint - preader = new PartitionReader(clusterId, new PartitionCheckpoint( - new HadoopStreamFile(DatabusStreamWaitingReader.getMinuteDirPath(streamDir, - CollectorStreamReader.getDateFromCollectorFile(TestUtil.files[0])), - "dummyfile", 0L), 20), fs, buffer, + prepareCheckpointList(new HadoopStreamFile(DatabusStreamWaitingReader. + getMinuteDirPath(streamDir, CollectorStreamReader. + getDateFromCollectorFile(TestUtil.files[0])), + "dummyfile", 0L), 20, partitionCheckpointList); + preader = new PartitionReader(clusterId, partitionCheckpointList, fs, buffer, streamDir, conf, inputFormatClass, null, - 1000, false, DataEncodingType.BASE64, prMetrics, true); + 1000, false, DataEncodingType.BASE64, prMetrics, true, partitionMinList); preader.init(); Assert.assertNotNull(preader.getReader()); Assert.assertEquals(preader.getReader().getClass().getName(), @@ -87,15 +105,16 @@ public void testInitialize() throws Exception { .getReader()).getReader().getClass().getName(), DatabusStreamWaitingReader.class.getName()); - //Read from startTime with checkpoint - preader = new PartitionReader(clusterId, new PartitionCheckpoint( - new HadoopStreamFile(DatabusStreamWaitingReader.getMinuteDirPath(streamDir, - CollectorStreamReader.getDateFromCollectorFile(TestUtil.files[0])), - "dummyfile", 0L), 20), fs, buffer, + //Read from startTime with checkpoint + prepareCheckpointList(new HadoopStreamFile(DatabusStreamWaitingReader. + getMinuteDirPath(streamDir, CollectorStreamReader. + getDateFromCollectorFile(TestUtil.files[0])), + "dummyfile", 0L), 20, partitionCheckpointList); + preader = new PartitionReader(clusterId, partitionCheckpointList, fs, buffer, streamDir, conf, inputFormatClass, CollectorStreamReader.getDateFromCollectorFile(TestUtil.files[0]), 1000, - false, DataEncodingType.BASE64, prMetrics, true); + false, DataEncodingType.BASE64, prMetrics, true, partitionMinList); preader.init(); Assert.assertNotNull(preader.getReader()); Assert.assertEquals(preader.getReader().getClass().getName(), @@ -105,4 +124,12 @@ public void testInitialize() throws Exception { DatabusStreamWaitingReader.class.getName()); } + public void prepareCheckpointList(StreamFile streamFile, int lineNum, + PartitionCheckpointList partitionCheckpointList) { + partitionCheckpointList = new PartitionCheckpointList(chkPoints); + Date date = DatabusStreamWaitingReader.getDateFromStreamDir(streamDir, + new Path(streamFile.toString())); + partitionCheckpointList.set(date.getMinutes(), new PartitionCheckpoint( + streamFile, lineNum)); + } } diff --git a/messaging-client-databus/src/test/java/com/inmobi/databus/partition/TestClusterReaderMultipleCollectors.java b/messaging-client-databus/src/test/java/com/inmobi/databus/partition/TestClusterReaderMultipleCollectors.java index 8c88b23..5ceeece 100644 --- a/messaging-client-databus/src/test/java/com/inmobi/databus/partition/TestClusterReaderMultipleCollectors.java +++ b/messaging-client-databus/src/test/java/com/inmobi/databus/partition/TestClusterReaderMultipleCollectors.java @@ -1,6 +1,11 @@ package com.inmobi.databus.partition; import java.io.IOException; +import java.util.Date; +import java.util.Map; +import java.util.Set; +import java.util.TreeMap; +import java.util.TreeSet; import java.util.concurrent.LinkedBlockingQueue; import org.apache.hadoop.conf.Configuration; @@ -13,6 +18,7 @@ import org.testng.annotations.Test; import com.inmobi.databus.Cluster; +import com.inmobi.databus.files.StreamFile; import com.inmobi.databus.readers.CollectorStreamReader; import com.inmobi.databus.readers.DatabusStreamWaitingReader; import com.inmobi.messaging.consumer.databus.DataEncodingType; @@ -41,10 +47,14 @@ public class TestClusterReaderMultipleCollectors { FileSystem fs; Path streamDir; Configuration conf = new Configuration(); + Set partitionMinList; + PartitionCheckpointList partitionCheckpointList; + int consumerNumber; @BeforeTest public void setup() throws Exception { // initialize config + consumerNumber = 1; fs = MiniClusterUtil.getDFSCluster(conf).getFileSystem(); cluster = TestUtil.setupDFSCluster(this.getClass().getSimpleName(), testStream, new PartitionId(clusterName, collectors[0]), @@ -54,6 +64,13 @@ testStream, new PartitionId(clusterName, collectors[0]), 1); streamDir = DatabusUtil.getStreamDir(StreamType.MERGED, new Path(cluster.getRootDir()), testStream); + Map chkpoints = new + TreeMap(); + partitionCheckpointList = new PartitionCheckpointList(chkpoints); + partitionMinList = new TreeSet(); + for (int i = 0; i < 60; i++) { + partitionMinList.add(i); + } } @AfterTest @@ -65,11 +82,11 @@ public void cleanup() throws IOException { @Test public void testReadFromStart() throws Exception { PartitionReaderStatsExposer prMetrics = new PartitionReaderStatsExposer( - testStream, "c1", partitionId.toString()); - preader = new PartitionReader(partitionId, null, fs, buffer, streamDir, - conf, TextInputFormat.class.getCanonicalName(), + testStream, "c1", partitionId.toString(), consumerNumber); + preader = new PartitionReader(partitionId, partitionCheckpointList, fs, + buffer, streamDir, conf, TextInputFormat.class.getCanonicalName(), CollectorStreamReader.getDateFromCollectorFile(files[0]), 1000, true, - DataEncodingType.BASE64, prMetrics, false); + DataEncodingType.BASE64, prMetrics, false, partitionMinList); preader.init(); Assert.assertTrue(buffer.isEmpty()); Assert.assertEquals(preader.getReader().getClass().getName(), @@ -158,14 +175,13 @@ public void testReadFromStart() throws Exception { Assert.assertTrue(prMetrics.getWaitTimeUnitsNewFile() > 0); prMetrics = new PartitionReaderStatsExposer( - testStream, "c1", partitionId.toString()); - preader = new PartitionReader(partitionId, new PartitionCheckpoint( - DatabusStreamWaitingReader.getHadoopStreamFile( - fs.getFileStatus(movedPath5)), 50), fs, buffer, streamDir, - conf, TextInputFormat.class.getCanonicalName(), - null, 1000, true, - DataEncodingType.BASE64, prMetrics, false); - + testStream, "c1", partitionId.toString(), consumerNumber); + prepareCheckpoint( DatabusStreamWaitingReader.getHadoopStreamFile( + fs.getFileStatus(movedPath5)), 50, movedPath5, partitionCheckpointList); + preader = new PartitionReader(partitionId, partitionCheckpointList, fs, + buffer, streamDir, conf, TextInputFormat.class.getCanonicalName(), + null, 1000, true, DataEncodingType.BASE64, prMetrics, false, + partitionMinList); preader.start(); TestUtil.assertBuffer(DatabusStreamWaitingReader.getHadoopStreamFile( fs.getFileStatus(movedPath5)), 4, 50, 50, partitionId, @@ -179,4 +195,12 @@ public void testReadFromStart() throws Exception { Assert.assertEquals(prMetrics.getMessagesAddedToBuffer(), 150); Assert.assertTrue(prMetrics.getCumulativeNanosForFetchMessage() > 0); } + + public void prepareCheckpoint(StreamFile streamFile, int lineNum, + Path databusFile, PartitionCheckpointList partitionCheckpointList) { + Date date = DatabusStreamWaitingReader.getDateFromStreamDir(streamDir, + databusFile.getParent()); + partitionCheckpointList.set(date.getMinutes(), new PartitionCheckpoint( + streamFile, lineNum)); + } } diff --git a/messaging-client-databus/src/test/java/com/inmobi/databus/partition/TestCollectorReader.java b/messaging-client-databus/src/test/java/com/inmobi/databus/partition/TestCollectorReader.java index f9f60c4..95147e5 100644 --- a/messaging-client-databus/src/test/java/com/inmobi/databus/partition/TestCollectorReader.java +++ b/messaging-client-databus/src/test/java/com/inmobi/databus/partition/TestCollectorReader.java @@ -49,10 +49,12 @@ public class TestCollectorReader { private Path streamsLocalDir; private Configuration conf = new Configuration(); private FileSystem fs; + int consumerNumber; @BeforeTest public void setup() throws Exception { // setup cluster + consumerNumber = 1; cluster = TestUtil.setupLocalCluster(this.getClass().getSimpleName(), testStream, partitionId, files, emptyfiles, databusFiles, 3); collectorDir = DatabusUtil.getCollectorStreamDir( @@ -71,7 +73,7 @@ public void cleanup() throws IOException { @Test public void testInitialize() throws Exception { CollectorReaderStatsExposer prMetrics = new CollectorReaderStatsExposer( - testStream, "c1", partitionId.toString()); + testStream, "c1", partitionId.toString(), consumerNumber); // Read from no where Throwable th = null; try { @@ -327,7 +329,7 @@ public void testInitialize() throws Exception { @Test public void testReadFromStart() throws Exception { CollectorReaderStatsExposer prMetrics = new CollectorReaderStatsExposer( - testStream, "c1", partitionId.toString()); + testStream, "c1", partitionId.toString(), consumerNumber); preader = new PartitionReader(partitionId, null, conf, fs, collectorDir, streamsLocalDir, buffer, testStream, CollectorStreamReader.getDateFromCollectorFile(files[0]), 10, 1000, @@ -379,7 +381,7 @@ public void testReadFromStart() throws Exception { @Test public void testReadFromCheckpointWithCollectorFileName() throws Exception { CollectorReaderStatsExposer prMetrics = new CollectorReaderStatsExposer( - testStream, "c1", partitionId.toString()); + testStream, "c1", partitionId.toString(), consumerNumber); preader = new PartitionReader(partitionId, new PartitionCheckpoint( CollectorStreamReader.getCollectorFile(files[1]), 20), conf, fs, collectorDir, streamsLocalDir, buffer, testStream, null, @@ -429,7 +431,7 @@ public void testReadFromCheckpointWithCollectorFileName() throws Exception { @Test public void testReadFromCheckpointWithLocalStreamFileName() throws Exception { CollectorReaderStatsExposer prMetrics = new CollectorReaderStatsExposer( - testStream, "c1", partitionId.toString()); + testStream, "c1", partitionId.toString(), consumerNumber); preader = new PartitionReader(partitionId, new PartitionCheckpoint( LocalStreamCollectorReader.getDatabusStreamFile(collectorName, files[1]), 20), @@ -480,7 +482,7 @@ public void testReadFromCheckpointWithLocalStreamFileName() throws Exception { @Test public void testReadFromCheckpointWithCollectorFile() throws Exception { CollectorReaderStatsExposer prMetrics = new CollectorReaderStatsExposer( - testStream, "c1", partitionId.toString()); + testStream, "c1", partitionId.toString(), consumerNumber); preader = new PartitionReader(partitionId, new PartitionCheckpoint( CollectorStreamReader.getCollectorFile(files[4]), 40), conf, fs, collectorDir, streamsLocalDir, buffer, testStream, null, @@ -519,11 +521,12 @@ public void testReadFromCheckpointWithCollectorFile() throws Exception { * Disable this test if partition reader should not read from start of stream * if check point does not exist. */ + @Test public void testReadFromCheckpointWithCollectorFileWhichDoesNotExist() throws Exception { CollectorReaderStatsExposer prMetrics = new CollectorReaderStatsExposer( - testStream, "c1", partitionId.toString()); + testStream, "c1", partitionId.toString(), consumerNumber); preader = new PartitionReader(partitionId, new PartitionCheckpoint( CollectorStreamReader.getCollectorFile(doesNotExist1), 40), conf, fs, collectorDir, streamsLocalDir, buffer, testStream, null, @@ -580,7 +583,7 @@ public void testReadFromCheckpointWithCollectorFileWhichDoesNotExist() public void testReadFromCheckpointWithLocalStreamFileWhichDoesNotExist() throws Exception { CollectorReaderStatsExposer prMetrics = new CollectorReaderStatsExposer( - testStream, "c1", partitionId.toString()); + testStream, "c1", partitionId.toString(), consumerNumber); preader = new PartitionReader(partitionId, new PartitionCheckpoint( LocalStreamCollectorReader.getDatabusStreamFile(collectorName, doesNotExist1), 20), @@ -633,7 +636,7 @@ public void testReadFromCheckpointWithLocalStreamFileWhichDoesNotExist() @Test public void testReadFromStartTimeInLocalStream() throws Exception { CollectorReaderStatsExposer prMetrics = new CollectorReaderStatsExposer( - testStream, "c1", partitionId.toString()); + testStream, "c1", partitionId.toString(), consumerNumber); preader = new PartitionReader(partitionId, new PartitionCheckpoint( CollectorStreamReader.getCollectorFile(files[0]), 20), conf, fs, collectorDir, streamsLocalDir, buffer, testStream, @@ -684,7 +687,7 @@ public void testReadFromStartTimeInLocalStream() throws Exception { @Test public void testReadFromStartTimeInLocalStream2() throws Exception { CollectorReaderStatsExposer prMetrics = new CollectorReaderStatsExposer( - testStream, "c1", partitionId.toString()); + testStream, "c1", partitionId.toString(), consumerNumber); preader = new PartitionReader(partitionId, new PartitionCheckpoint( CollectorStreamReader.getCollectorFile(files[0]), 20), conf, fs, collectorDir, streamsLocalDir, buffer, testStream, @@ -733,7 +736,7 @@ public void testReadFromStartTimeInLocalStream2() throws Exception { @Test public void testReadFromStartTimeInCollectorStream() throws Exception { CollectorReaderStatsExposer prMetrics = new CollectorReaderStatsExposer( - testStream, "c1", partitionId.toString()); + testStream, "c1", partitionId.toString(), consumerNumber); preader = new PartitionReader(partitionId, new PartitionCheckpoint( CollectorStreamReader.getCollectorFile(files[0]), 20), conf, fs, collectorDir, streamsLocalDir, buffer, testStream, @@ -772,7 +775,7 @@ public void testReadFromStartTimeInCollectorStream() throws Exception { @Test public void testReadFromStartTimeInCollectorStream2() throws Exception { CollectorReaderStatsExposer prMetrics = new CollectorReaderStatsExposer( - testStream, "c1", partitionId.toString()); + testStream, "c1", partitionId.toString(), consumerNumber); preader = new PartitionReader(partitionId, new PartitionCheckpoint( CollectorStreamReader.getCollectorFile(files[0]), 20), conf, fs, collectorDir, streamsLocalDir, buffer, testStream, @@ -811,7 +814,7 @@ public void testReadFromStartTimeInCollectorStream2() throws Exception { @Test public void testReadFromStartTimeBeforeStream() throws Exception { CollectorReaderStatsExposer prMetrics = new CollectorReaderStatsExposer( - testStream, "c1", partitionId.toString()); + testStream, "c1", partitionId.toString(), consumerNumber); preader = new PartitionReader(partitionId, new PartitionCheckpoint( CollectorStreamReader.getCollectorFile(files[1]), 20), conf, fs, collectorDir, streamsLocalDir, buffer, testStream, @@ -864,7 +867,7 @@ public void testReadFromStartTimeBeforeStream() throws Exception { @Test public void testReadFromStartTimeAfterStream() throws Exception { CollectorReaderStatsExposer prMetrics = new CollectorReaderStatsExposer( - testStream, "c1", partitionId.toString()); + testStream, "c1", partitionId.toString(), consumerNumber); preader = new PartitionReader(partitionId, new PartitionCheckpoint( CollectorStreamReader.getCollectorFile(files[1]), 20), conf, fs, collectorDir, streamsLocalDir, buffer, testStream, @@ -886,5 +889,5 @@ public void testReadFromStartTimeAfterStream() throws Exception { Assert.assertEquals(prMetrics.getSwitchesFromCollectorToLocal(), 0); Assert.assertEquals(prMetrics.getSwitchesFromLocalToCollector(), 0); Assert.assertEquals(prMetrics.getCumulativeNanosForFetchMessage(), 0); - } + } } diff --git a/messaging-client-databus/src/test/java/com/inmobi/databus/partition/TestCollectorReaderEmptyStream.java b/messaging-client-databus/src/test/java/com/inmobi/databus/partition/TestCollectorReaderEmptyStream.java index 1cb613e..a0c808c 100644 --- a/messaging-client-databus/src/test/java/com/inmobi/databus/partition/TestCollectorReaderEmptyStream.java +++ b/messaging-client-databus/src/test/java/com/inmobi/databus/partition/TestCollectorReaderEmptyStream.java @@ -40,10 +40,12 @@ public class TestCollectorReaderEmptyStream { private Path streamsLocalDir; private Configuration conf = new Configuration(); private FileSystem fs; + int consumerNumber; @BeforeTest public void setup() throws Exception { // setup cluster + consumerNumber = 1; cluster = TestUtil.setupLocalCluster(this.getClass().getSimpleName(), testStream, partitionId, null, null, 0); collectorDir = DatabusUtil.getCollectorStreamDir( @@ -63,7 +65,7 @@ public void cleanup() throws IOException { @Test public void testInitialize() throws Exception { CollectorReaderStatsExposer prMetrics = new CollectorReaderStatsExposer( - testStream, "c1", partitionId.toString()); + testStream, "c1", partitionId.toString(), consumerNumber); // Read from start time preader = new PartitionReader(partitionId, null, conf, fs, diff --git a/messaging-client-databus/src/test/java/com/inmobi/databus/partition/TestCollectorStreamWithEmptyFiles.java b/messaging-client-databus/src/test/java/com/inmobi/databus/partition/TestCollectorStreamWithEmptyFiles.java index 6833ff1..f9afbda 100644 --- a/messaging-client-databus/src/test/java/com/inmobi/databus/partition/TestCollectorStreamWithEmptyFiles.java +++ b/messaging-client-databus/src/test/java/com/inmobi/databus/partition/TestCollectorStreamWithEmptyFiles.java @@ -38,10 +38,12 @@ public class TestCollectorStreamWithEmptyFiles { private Path streamsLocalDir; private Configuration conf = new Configuration(); private FileSystem fs; + int consumerNumber; @BeforeTest public void setup() throws Exception { // setup cluster + consumerNumber = 1; cluster = TestUtil.setupLocalCluster(this.getClass().getSimpleName(), testStream, partitionId, files, emptyfiles, 0); collectorDir = DatabusUtil.getCollectorStreamDir( @@ -60,7 +62,7 @@ public void cleanup() throws IOException { @Test public void testReadFromStart() throws Exception { CollectorReaderStatsExposer prMetrics = new CollectorReaderStatsExposer( - testStream, "c1", partitionId.toString()); + testStream, "c1", partitionId.toString(), consumerNumber); PartitionReader preader = new PartitionReader(partitionId, null, conf, fs, collectorDir, streamsLocalDir, buffer, testStream, CollectorStreamReader.getDateFromCollectorFile(files[0]), 5, 1000, diff --git a/messaging-client-databus/src/test/java/com/inmobi/databus/partition/TestCurrentFile.java b/messaging-client-databus/src/test/java/com/inmobi/databus/partition/TestCurrentFile.java index 0fd1f94..743c1f2 100644 --- a/messaging-client-databus/src/test/java/com/inmobi/databus/partition/TestCurrentFile.java +++ b/messaging-client-databus/src/test/java/com/inmobi/databus/partition/TestCurrentFile.java @@ -44,6 +44,7 @@ public class TestCurrentFile { private String currentScribeFile = TestUtil.files[3]; Configuration conf = new Configuration(); private Path streamsLocalDir; + int consumerNumber; private void writeMessages(FSDataOutputStream out, int num) @@ -65,6 +66,7 @@ public void cleanup() throws IOException { @BeforeTest public void setup() throws Exception { + consumerNumber = 1; cluster = TestUtil.setupDFSCluster(this.getClass().getSimpleName(), testStream, partitionId, MiniClusterUtil.getDFSCluster(conf).getFileSystem().getUri().toString(), @@ -80,7 +82,7 @@ public void setup() throws Exception { @Test public void testReadFromCurrentScribeFile() throws Exception { CollectorReaderStatsExposer prMetrics = new CollectorReaderStatsExposer( - testStream, "c1", partitionId.toString()); + testStream, "c1", partitionId.toString(), consumerNumber); preader = new PartitionReader(partitionId, null, conf, fs, collectorDir, streamsLocalDir, buffer, testStream, CollectorStreamReader.getDateFromCollectorFile(currentScribeFile), 1000, diff --git a/messaging-client-databus/src/test/java/com/inmobi/databus/partition/TestPartitionReaderCollectorStream.java b/messaging-client-databus/src/test/java/com/inmobi/databus/partition/TestPartitionReaderCollectorStream.java index 415ba4d..9c94c6f 100644 --- a/messaging-client-databus/src/test/java/com/inmobi/databus/partition/TestPartitionReaderCollectorStream.java +++ b/messaging-client-databus/src/test/java/com/inmobi/databus/partition/TestPartitionReaderCollectorStream.java @@ -46,10 +46,11 @@ public class TestPartitionReaderCollectorStream { private Path streamsLocalDir; private Configuration conf = new Configuration(); private FileSystem fs; - + int consumerNUmber; @BeforeTest public void setup() throws Exception { + consumerNUmber = 1; // setup cluster cluster = TestUtil.setupLocalCluster(this.getClass().getSimpleName(), testStream, partitionId, files, null, 0); @@ -69,7 +70,7 @@ public void cleanup() throws IOException { @Test public void testInitialize() throws Exception { CollectorReaderStatsExposer prMetrics = new CollectorReaderStatsExposer( - testStream, "c1", partitionId.toString()); + testStream, "c1", partitionId.toString(), consumerNUmber); // Read from start preader = new PartitionReader(partitionId, null, conf, fs, collectorDir, streamsLocalDir, @@ -290,7 +291,7 @@ public void testInitialize() throws Exception { @Test public void testReadFromStart() throws Exception { CollectorReaderStatsExposer prMetrics = new CollectorReaderStatsExposer( - testStream, "c1", partitionId.toString()); + testStream, "c1", partitionId.toString(), consumerNUmber); preader = new PartitionReader(partitionId, null, conf, fs, collectorDir, streamsLocalDir, buffer, testStream, CollectorStreamReader.getDateFromCollectorFile(files[0]), 1000, 1000, @@ -329,7 +330,7 @@ public void testReadFromStart() throws Exception { @Test public void testReadFromCheckpoint() throws Exception { CollectorReaderStatsExposer prMetrics = new CollectorReaderStatsExposer( - testStream, "c1", partitionId.toString()); + testStream, "c1", partitionId.toString(), consumerNUmber); preader = new PartitionReader(partitionId, new PartitionCheckpoint( CollectorStreamReader.getCollectorFile(files[1]), 20), conf, fs, collectorDir, streamsLocalDir, buffer, testStream, null, @@ -366,7 +367,7 @@ public void testReadFromCheckpoint() throws Exception { @Test public void testReadFromCheckpointWhichDoesNotExist() throws Exception { CollectorReaderStatsExposer prMetrics = new CollectorReaderStatsExposer( - testStream, "c1", partitionId.toString()); + testStream, "c1", partitionId.toString(), consumerNUmber); preader = new PartitionReader(partitionId, new PartitionCheckpoint( CollectorStreamReader.getCollectorFile(doesNotExist1), 20), conf, fs, collectorDir, streamsLocalDir, buffer, testStream, @@ -405,7 +406,7 @@ public void testReadFromCheckpointWhichDoesNotExist() throws Exception { @Test public void testReadFromCheckpointWhichDoesNotExist2() throws Exception { CollectorReaderStatsExposer prMetrics = new CollectorReaderStatsExposer( - testStream, "c1", partitionId.toString()); + testStream, "c1", partitionId.toString(), consumerNUmber); Throwable th = null; try { preader = new PartitionReader(partitionId, new PartitionCheckpoint( @@ -423,7 +424,7 @@ public void testReadFromCheckpointWhichDoesNotExist2() throws Exception { @Test public void testReadFromCheckpointWhichDoesNotExist3() throws Exception { CollectorReaderStatsExposer prMetrics = new CollectorReaderStatsExposer( - testStream, "c1", partitionId.toString()); + testStream, "c1", partitionId.toString(), consumerNUmber); Throwable th = null; try { preader = new PartitionReader(partitionId, new PartitionCheckpoint( @@ -441,7 +442,7 @@ public void testReadFromCheckpointWhichDoesNotExist3() throws Exception { @Test public void testReadFromStartTime() throws Exception { CollectorReaderStatsExposer prMetrics = new CollectorReaderStatsExposer( - testStream, "c1", partitionId.toString()); + testStream, "c1", partitionId.toString(), consumerNUmber); preader = new PartitionReader(partitionId, new PartitionCheckpoint( CollectorStreamReader.getCollectorFile(files[1]), 20), conf, fs, collectorDir, streamsLocalDir, buffer, testStream, @@ -479,7 +480,7 @@ public void testReadFromStartTime() throws Exception { @Test public void testReadFromStartTimeWithinStream() throws Exception { CollectorReaderStatsExposer prMetrics = new CollectorReaderStatsExposer( - testStream, "c1", partitionId.toString()); + testStream, "c1", partitionId.toString(), consumerNUmber); preader = new PartitionReader(partitionId, new PartitionCheckpoint( CollectorStreamReader.getCollectorFile(files[1]), 20), conf, fs, collectorDir, streamsLocalDir, buffer, testStream, @@ -517,7 +518,7 @@ public void testReadFromStartTimeWithinStream() throws Exception { @Test public void testReadFromStartTimeBeforeStream() throws Exception { CollectorReaderStatsExposer prMetrics = new CollectorReaderStatsExposer( - testStream, "c1", partitionId.toString()); + testStream, "c1", partitionId.toString(), consumerNUmber); preader = new PartitionReader(partitionId, new PartitionCheckpoint( CollectorStreamReader.getCollectorFile(files[1]), 20), conf, fs, collectorDir, streamsLocalDir, buffer, testStream, @@ -557,7 +558,7 @@ public void testReadFromStartTimeBeforeStream() throws Exception { @Test public void testReadFromStartTimeAfterStream() throws Exception { CollectorReaderStatsExposer prMetrics = new CollectorReaderStatsExposer( - testStream, "c1", partitionId.toString()); + testStream, "c1", partitionId.toString(), consumerNUmber); preader = new PartitionReader(partitionId, new PartitionCheckpoint( CollectorStreamReader.getCollectorFile(files[1]), 20), conf, fs, collectorDir, streamsLocalDir, buffer, testStream, diff --git a/messaging-client-databus/src/test/java/com/inmobi/databus/partition/TestPartitionReaderHadoopStream.java b/messaging-client-databus/src/test/java/com/inmobi/databus/partition/TestPartitionReaderHadoopStream.java index 5df5808..081af1c 100644 --- a/messaging-client-databus/src/test/java/com/inmobi/databus/partition/TestPartitionReaderHadoopStream.java +++ b/messaging-client-databus/src/test/java/com/inmobi/databus/partition/TestPartitionReaderHadoopStream.java @@ -1,6 +1,9 @@ package com.inmobi.databus.partition; import java.io.IOException; +import java.util.Map; +import java.util.TreeMap; +import java.util.TreeSet; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -16,6 +19,7 @@ public class TestPartitionReaderHadoopStream extends TestAbstractClusterReader { @BeforeTest public void setup() throws Exception { + consumerNumber = 1; // setup fs files = new String[] {HadoopUtil.files[1], HadoopUtil.files[3], HadoopUtil.files[5]}; @@ -25,53 +29,76 @@ public void setup() throws Exception { HadoopUtil.setupHadoopCluster(conf, files, null, databusFiles, streamDir); inputFormatClass = SequenceFileInputFormat.class.getName(); dataEncoding = DataEncodingType.NONE; + partitionMinList = new TreeSet(); + for (int i =0; i < 60; i++) { + partitionMinList.add(i); + } + Map chkpoints = new TreeMap(); + partitionCheckpointList = new PartitionCheckpointList(chkpoints); } @AfterTest public void cleanup() throws IOException { fs.delete(streamDir.getParent(), true); } - + @Test public void testInitialize() throws Exception { super.testInitialize(); } - + @Test public void testReadFromStart() throws Exception { super.testReadFromStart(); } - + @Test public void testReadFromCheckpoint() throws Exception { super.testReadFromCheckpoint(); } - + @Test public void testReadFromCheckpointWhichDoesNotExist() throws Exception { super.testReadFromCheckpointWhichDoesNotExist(); - } - + } + @Test public void testReadFromStartTime() throws Exception { super.testReadFromStartTime(); - } - + } + @Test public void testReadFromStartTimeWithinStream() throws Exception { super.testReadFromStartTimeWithinStream(); - } - + } + + @Test public void testReadFromStartTimeBeforeStream() throws Exception { super.testReadFromStartTimeBeforeStream(); - } + } @Test public void testReadFromStartTimeAfterStream() throws Exception { super.testReadFromStartTimeAfterStream(); + } + + @Test + public void testReadFromCheckpointWithSingleMinute() throws Exception { + super.testReadFromCheckpointWithSingleMinute(); } - + + @Test + public void testReadFromCheckpointMultipleMinutes() throws Exception { + super.testReadFromCheckpointMultipleMinutes(); + } + + @Test + public void testReadFromCheckpointSomeMinutes() throws Exception { + super.testReadFromCheckpointSomeMinutes(); + } + @Override Path getStreamsDir() { return streamDir; diff --git a/messaging-client-databus/src/test/java/com/inmobi/databus/partition/TestPartitionReaderLocalCollectorStream.java b/messaging-client-databus/src/test/java/com/inmobi/databus/partition/TestPartitionReaderLocalCollectorStream.java index 7a4aa2f..8ac7655 100644 --- a/messaging-client-databus/src/test/java/com/inmobi/databus/partition/TestPartitionReaderLocalCollectorStream.java +++ b/messaging-client-databus/src/test/java/com/inmobi/databus/partition/TestPartitionReaderLocalCollectorStream.java @@ -45,9 +45,11 @@ public class TestPartitionReaderLocalCollectorStream { private Path streamsLocalDir; private Configuration conf = new Configuration(); private FileSystem fs; + int consumerNumber; @BeforeTest public void setup() throws Exception { + consumerNumber = 1; // setup cluster cluster = TestUtil.setupLocalCluster(this.getClass().getSimpleName(), testStream, partitionId, files, null, databusFiles, 3); @@ -67,7 +69,7 @@ public void cleanup() throws IOException { @Test public void testInitialize() throws Exception { CollectorReaderStatsExposer prMetrics = new CollectorReaderStatsExposer( - testStream, "c1", partitionId.toString()); + testStream, "c1", partitionId.toString(), consumerNumber); // Read from starttime of stream preader = new PartitionReader(partitionId, null, conf, fs, collectorDir, streamsLocalDir, buffer, testStream, @@ -213,7 +215,7 @@ public void testInitialize() throws Exception { @Test public void testReadFromStart() throws Exception { CollectorReaderStatsExposer prMetrics = new CollectorReaderStatsExposer( - testStream, "c1", partitionId.toString()); + testStream, "c1", partitionId.toString(), consumerNumber); preader = new PartitionReader(partitionId, null, conf, fs, collectorDir, streamsLocalDir, buffer, testStream, CollectorStreamReader.getDateFromCollectorFile(files[0]), 10, 1000, @@ -252,7 +254,7 @@ public void testReadFromStart() throws Exception { @Test public void testReadFromCheckpointWithCollectorFileName() throws Exception { CollectorReaderStatsExposer prMetrics = new CollectorReaderStatsExposer( - testStream, "c1", partitionId.toString()); + testStream, "c1", partitionId.toString(), consumerNumber); preader = new PartitionReader(partitionId, new PartitionCheckpoint( CollectorStreamReader.getCollectorFile(files[1]), 20), conf, fs, collectorDir, streamsLocalDir, buffer, testStream, null, @@ -289,7 +291,7 @@ public void testReadFromCheckpointWithCollectorFileName() throws Exception { @Test public void testReadFromCheckpointWithLocalStreamFileName() throws Exception { CollectorReaderStatsExposer prMetrics = new CollectorReaderStatsExposer( - testStream, "c1", partitionId.toString()); + testStream, "c1", partitionId.toString(), consumerNumber); preader = new PartitionReader(partitionId, new PartitionCheckpoint( LocalStreamCollectorReader.getDatabusStreamFile(collectorName, files[1]), 20), @@ -332,7 +334,7 @@ public void testReadFromCheckpointWithLocalStreamFileName() throws Exception { public void testReadFromCheckpointWithCollectorFileWhichDoesNotExist() throws Exception { CollectorReaderStatsExposer prMetrics = new CollectorReaderStatsExposer( - testStream, "c1", partitionId.toString()); + testStream, "c1", partitionId.toString(), consumerNumber); preader = new PartitionReader(partitionId, new PartitionCheckpoint( CollectorStreamReader.getCollectorFile(doesNotExist1), 40), conf, fs, collectorDir, streamsLocalDir, buffer, testStream, null, @@ -376,7 +378,7 @@ public void testReadFromCheckpointWithCollectorFileWhichDoesNotExist() public void testReadFromCheckpointWithLocalStreamFileWhichDoesNotExist() throws Exception { CollectorReaderStatsExposer prMetrics = new CollectorReaderStatsExposer( - testStream, "c1", partitionId.toString()); + testStream, "c1", partitionId.toString(), consumerNumber); preader = new PartitionReader(partitionId, new PartitionCheckpoint( LocalStreamCollectorReader.getDatabusStreamFile(collectorName, doesNotExist1), 20), @@ -417,7 +419,7 @@ public void testReadFromCheckpointWithLocalStreamFileWhichDoesNotExist() @Test public void testReadFromStartTimeInLocalStream() throws Exception { CollectorReaderStatsExposer prMetrics = new CollectorReaderStatsExposer( - testStream, "c1", partitionId.toString()); + testStream, "c1", partitionId.toString(), consumerNumber); preader = new PartitionReader(partitionId, new PartitionCheckpoint( CollectorStreamReader.getCollectorFile(files[0]), 20), conf, fs, collectorDir, streamsLocalDir, buffer, testStream, @@ -455,7 +457,7 @@ public void testReadFromStartTimeInLocalStream() throws Exception { @Test public void testReadFromStartTimeInLocalStream2() throws Exception { CollectorReaderStatsExposer prMetrics = new CollectorReaderStatsExposer( - testStream, "c1", partitionId.toString()); + testStream, "c1", partitionId.toString(), consumerNumber); preader = new PartitionReader(partitionId, new PartitionCheckpoint( CollectorStreamReader.getCollectorFile(files[0]), 20), conf, fs, collectorDir, streamsLocalDir, buffer, testStream, @@ -491,7 +493,7 @@ public void testReadFromStartTimeInLocalStream2() throws Exception { @Test public void testReadFromStartTimeBeforeStream() throws Exception { CollectorReaderStatsExposer prMetrics = new CollectorReaderStatsExposer( - testStream, "c1", partitionId.toString()); + testStream, "c1", partitionId.toString(), consumerNumber); preader = new PartitionReader(partitionId, new PartitionCheckpoint( CollectorStreamReader.getCollectorFile(files[1]), 20), conf, fs, collectorDir, streamsLocalDir, buffer, testStream, @@ -531,7 +533,7 @@ public void testReadFromStartTimeBeforeStream() throws Exception { @Test public void testReadFromStartTimeAfterStream() throws Exception { CollectorReaderStatsExposer prMetrics = new CollectorReaderStatsExposer( - testStream, "c1", partitionId.toString()); + testStream, "c1", partitionId.toString(), consumerNumber); preader = new PartitionReader(partitionId, new PartitionCheckpoint( CollectorStreamReader.getCollectorFile(files[1]), 20), conf, fs, collectorDir, streamsLocalDir, buffer, testStream, diff --git a/messaging-client-databus/src/test/java/com/inmobi/databus/partition/TestPartitionReaderLocalStream.java b/messaging-client-databus/src/test/java/com/inmobi/databus/partition/TestPartitionReaderLocalStream.java index f260027..a09a7e8 100644 --- a/messaging-client-databus/src/test/java/com/inmobi/databus/partition/TestPartitionReaderLocalStream.java +++ b/messaging-client-databus/src/test/java/com/inmobi/databus/partition/TestPartitionReaderLocalStream.java @@ -19,6 +19,7 @@ public class TestPartitionReaderLocalStream extends TestAbstractClusterReader { Cluster cluster; @BeforeTest public void setup() throws Exception { + consumerNumber = 1; // setup cluster cluster = TestUtil.setupLocalCluster(this.getClass().getSimpleName(), testStream, new PartitionId(clusterName, collectorName), files, null, diff --git a/messaging-client-databus/src/test/java/com/inmobi/databus/partition/TestPartitionReaderMergedStream.java b/messaging-client-databus/src/test/java/com/inmobi/databus/partition/TestPartitionReaderMergedStream.java index fe45c08..61d4fda 100644 --- a/messaging-client-databus/src/test/java/com/inmobi/databus/partition/TestPartitionReaderMergedStream.java +++ b/messaging-client-databus/src/test/java/com/inmobi/databus/partition/TestPartitionReaderMergedStream.java @@ -19,6 +19,7 @@ public class TestPartitionReaderMergedStream extends TestAbstractClusterReader { Cluster cluster; @BeforeTest public void setup() throws Exception { + consumerNumber = 1; // setup cluster cluster = TestUtil.setupLocalCluster(this.getClass().getSimpleName(), testStream, new PartitionId(clusterName, collectorName), files, null, diff --git a/messaging-client-databus/src/test/java/com/inmobi/databus/partition/TestPartitionReaderMovingFilesFromCollectorStream.java b/messaging-client-databus/src/test/java/com/inmobi/databus/partition/TestPartitionReaderMovingFilesFromCollectorStream.java index 2c5c8c1..107e4d8 100644 --- a/messaging-client-databus/src/test/java/com/inmobi/databus/partition/TestPartitionReaderMovingFilesFromCollectorStream.java +++ b/messaging-client-databus/src/test/java/com/inmobi/databus/partition/TestPartitionReaderMovingFilesFromCollectorStream.java @@ -47,10 +47,12 @@ public class TestPartitionReaderMovingFilesFromCollectorStream { private String[] emptyfiles = new String[] {TestUtil.files[1], TestUtil.files[5], TestUtil.files[7]}; + int consumerNumber; @BeforeTest public void setup() throws Exception { // setup cluster + consumerNumber = 1; cluster = TestUtil.setupLocalCluster(this.getClass().getSimpleName(), testStream, partitionId, files, emptyfiles, 1); collectorDir = DatabusUtil.getCollectorStreamDir( @@ -69,7 +71,7 @@ public void cleanup() throws IOException { @Test public void testCollectorFileMoved() throws Exception { CollectorReaderStatsExposer prMetrics = new CollectorReaderStatsExposer( - testStream, "c1", partitionId.toString()); + testStream, "c1", partitionId.toString(), consumerNumber); preader = new PartitionReader(partitionId, null, conf, fs, collectorDir, streamsLocalDir, buffer, testStream, CollectorStreamReader.getDateFromCollectorFile(files[0]), diff --git a/messaging-client-databus/src/test/java/com/inmobi/databus/partition/TestPartitionReaderMovingFilesFromLocalStream.java b/messaging-client-databus/src/test/java/com/inmobi/databus/partition/TestPartitionReaderMovingFilesFromLocalStream.java index 4b8dd12..05202a5 100644 --- a/messaging-client-databus/src/test/java/com/inmobi/databus/partition/TestPartitionReaderMovingFilesFromLocalStream.java +++ b/messaging-client-databus/src/test/java/com/inmobi/databus/partition/TestPartitionReaderMovingFilesFromLocalStream.java @@ -44,10 +44,12 @@ public class TestPartitionReaderMovingFilesFromLocalStream { TestUtil.files[5], TestUtil.files[6], TestUtil.files[7], TestUtil.files[8]}; private Path[] databusFiles = new Path[8]; + int consumerNumber; @BeforeTest public void setup() throws Exception { // setup cluster + consumerNumber = 1; cluster = TestUtil.setupLocalCluster(this.getClass().getSimpleName(), testStream, partitionId, files, null, databusFiles, 4); collectorDir = DatabusUtil.getCollectorStreamDir( @@ -66,7 +68,7 @@ public void cleanup() throws IOException { @Test public void testLocalStreamFileMoved() throws Exception { CollectorReaderStatsExposer prMetrics = new CollectorReaderStatsExposer( - testStream, "c1", partitionId.toString()); + testStream, "c1", partitionId.toString(), consumerNumber); preader = new PartitionReader(partitionId, null, conf, fs, collectorDir, streamsLocalDir, buffer, testStream, CollectorStreamReader.getDateFromCollectorFile(files[0]), diff --git a/messaging-client-databus/src/test/java/com/inmobi/databus/partition/TestPartitionReaderWaitingHadoopStream.java b/messaging-client-databus/src/test/java/com/inmobi/databus/partition/TestPartitionReaderWaitingHadoopStream.java index 6e0d3f4..e8c149d 100644 --- a/messaging-client-databus/src/test/java/com/inmobi/databus/partition/TestPartitionReaderWaitingHadoopStream.java +++ b/messaging-client-databus/src/test/java/com/inmobi/databus/partition/TestPartitionReaderWaitingHadoopStream.java @@ -1,6 +1,9 @@ package com.inmobi.databus.partition; import java.io.IOException; +import java.util.Map; +import java.util.TreeMap; +import java.util.TreeSet; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; @@ -18,6 +21,7 @@ public class TestPartitionReaderWaitingHadoopStream extends @BeforeTest public void setup() throws Exception { + consumerNumber = 1; conf = new Configuration(); files = new String[] {HadoopUtil.files[1], HadoopUtil.files[3], HadoopUtil.files[5]}; @@ -30,6 +34,13 @@ public void setup() throws Exception { HadoopUtil.setupHadoopCluster(conf, files, null, databusFiles, streamDir); inputFormatClass = SequenceFileInputFormat.class.getName(); dataEncoding = DataEncodingType.NONE; + partitionMinList = new TreeSet(); + for (int i =0; i< 60; i++) { + partitionMinList.add(i); + } + Map list = new + TreeMap(); + partitionCheckpointlist = new PartitionCheckpointList(list); } @AfterTest diff --git a/messaging-client-databus/src/test/java/com/inmobi/databus/partition/TestPartitionReaderWaitingLocalStream.java b/messaging-client-databus/src/test/java/com/inmobi/databus/partition/TestPartitionReaderWaitingLocalStream.java index 2d7de32..db9dfc1 100644 --- a/messaging-client-databus/src/test/java/com/inmobi/databus/partition/TestPartitionReaderWaitingLocalStream.java +++ b/messaging-client-databus/src/test/java/com/inmobi/databus/partition/TestPartitionReaderWaitingLocalStream.java @@ -1,6 +1,9 @@ package com.inmobi.databus.partition; import java.io.IOException; +import java.util.Map; +import java.util.TreeMap; +import java.util.TreeSet; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -33,6 +36,14 @@ testStream, new PartitionId(clusterName, collectorName), files, null, fs = FileSystem.get(conf); streamDir = DatabusUtil.getStreamDir(StreamType.LOCAL, new Path(cluster.getRootDir()), testStream); + partitionMinList = new TreeSet(); + for (int i =0; i< 60; i++) { + partitionMinList.add(i); + } + Map list = new + TreeMap(); + partitionCheckpointlist = new PartitionCheckpointList(list); + consumerNumber = 1; } void setupFiles(String[] files, Path[] newDatabusFiles) throws Exception { diff --git a/messaging-client-databus/src/test/java/com/inmobi/databus/partition/TestPartitionReaderWaitingMergeStream.java b/messaging-client-databus/src/test/java/com/inmobi/databus/partition/TestPartitionReaderWaitingMergeStream.java index 6afb10c..09d66f1 100644 --- a/messaging-client-databus/src/test/java/com/inmobi/databus/partition/TestPartitionReaderWaitingMergeStream.java +++ b/messaging-client-databus/src/test/java/com/inmobi/databus/partition/TestPartitionReaderWaitingMergeStream.java @@ -1,6 +1,9 @@ package com.inmobi.databus.partition; import java.io.IOException; +import java.util.Map; +import java.util.TreeMap; +import java.util.TreeSet; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -33,6 +36,12 @@ testStream, new PartitionId(clusterName, collectorName), files, null, fs = FileSystem.get(conf); streamDir = DatabusUtil.getStreamDir(StreamType.MERGED, new Path(cluster.getRootDir()), testStream); + partitionMinList = new TreeSet(); + for (int i =0; i< 60; i++) { + partitionMinList.add(i); + } + Map list = new TreeMap(); + partitionCheckpointlist = new PartitionCheckpointList(list); } void setupFiles(String[] files, Path[] newDatabusFiles) throws Exception { diff --git a/messaging-client-databus/src/test/java/com/inmobi/databus/readers/TestAbstractDatabusWaitingReader.java b/messaging-client-databus/src/test/java/com/inmobi/databus/readers/TestAbstractDatabusWaitingReader.java index 8ac2e8c..51a75d7 100644 --- a/messaging-client-databus/src/test/java/com/inmobi/databus/readers/TestAbstractDatabusWaitingReader.java +++ b/messaging-client-databus/src/test/java/com/inmobi/databus/readers/TestAbstractDatabusWaitingReader.java @@ -2,6 +2,9 @@ import java.io.IOException; import java.util.Calendar; +import java.util.Map; +import java.util.Set; +import java.util.TreeMap; import org.apache.commons.codec.binary.Base64; import org.apache.hadoop.conf.Configuration; @@ -10,6 +13,7 @@ import org.apache.hadoop.io.Text; import org.testng.Assert; import com.inmobi.databus.partition.PartitionCheckpoint; +import com.inmobi.databus.partition.PartitionCheckpointList; import com.inmobi.databus.partition.PartitionId; import com.inmobi.messaging.consumer.util.HadoopUtil; import com.inmobi.messaging.consumer.util.MessageUtil; @@ -31,6 +35,10 @@ public abstract class TestAbstractDatabusWaitingReader { protected Path streamDir; protected String inputFormatClass; protected boolean encoded; + public Set partitionMinList; + public PartitionCheckpointList partitionCheckpointList; + Map chkPoints; + int consumerNumber; public void cleanup() throws IOException { } @@ -39,11 +47,12 @@ public void cleanup() throws IOException { public void testInitialize() throws Exception { PartitionReaderStatsExposer metrics = new PartitionReaderStatsExposer( - testStream, "c1", partitionId.toString()); + testStream, "c1", partitionId.toString(), consumerNumber); // Read from start lreader = new DatabusStreamWaitingReader(partitionId, fs, streamDir, - inputFormatClass, conf, 1000, metrics, false); + inputFormatClass, conf, 1000, metrics, false, partitionMinList, + partitionCheckpointList); Calendar cal = Calendar.getInstance(); cal.setTime(DatabusStreamWaitingReader.getDateFromStreamDir(streamDir, finalFiles[0].getParent())); @@ -124,11 +133,13 @@ static void readFile(StreamReader reader, int fileNum, public void testReadFromStart() throws Exception { + initializePartitionCheckpointList(); PartitionReaderStatsExposer metrics = new PartitionReaderStatsExposer( - testStream, "c1", partitionId.toString()); + testStream, "c1", partitionId.toString(), consumerNumber); lreader = new DatabusStreamWaitingReader(partitionId, fs, getStreamsDir(), - inputFormatClass, conf , 1000, metrics, false); + inputFormatClass, conf , 1000, metrics, false, partitionMinList, + partitionCheckpointList); lreader.build(DatabusStreamWaitingReader.getDateFromStreamDir(streamDir, finalFiles[0].getParent())); lreader.initFromStart(); @@ -147,10 +158,12 @@ fs, getStreamsDir(), } public void testReadFromCheckpoint() throws Exception { + initializePartitionCheckpointList(); PartitionReaderStatsExposer metrics = new PartitionReaderStatsExposer( - testStream, "c1", partitionId.toString()); + testStream, "c1", partitionId.toString(), consumerNumber); lreader = new DatabusStreamWaitingReader(partitionId, - fs, getStreamsDir(), inputFormatClass, conf, 1000, metrics, false); + fs, getStreamsDir(), inputFormatClass, conf, 1000, metrics, false, + partitionMinList, partitionCheckpointList); PartitionCheckpoint pcp = new PartitionCheckpoint( DatabusStreamWaitingReader.getHadoopStreamFile( fs.getFileStatus( finalFiles[1])), 20); @@ -169,10 +182,12 @@ public void testReadFromCheckpoint() throws Exception { } public void testReadFromTimeStamp() throws Exception { - PartitionReaderStatsExposer metrics = new PartitionReaderStatsExposer( - testStream, "c1", partitionId.toString()); + initializePartitionCheckpointList(); + PartitionReaderStatsExposer metrics = new PartitionReaderStatsExposer( + testStream, "c1", partitionId.toString(), consumerNumber); lreader = new DatabusStreamWaitingReader(partitionId, - fs, getStreamsDir(), inputFormatClass, conf, 1000, metrics, false); + fs, getStreamsDir(), inputFormatClass, conf, 1000, metrics, false, + partitionMinList, partitionCheckpointList); lreader.build(DatabusStreamWaitingReader.getDateFromStreamDir(streamDir, finalFiles[1].getParent())); lreader.initializeCurrentFile( @@ -189,5 +204,8 @@ public void testReadFromTimeStamp() throws Exception { Assert.assertEquals(metrics.getWaitTimeUnitsNewFile(), 0); Assert.assertTrue(metrics.getCumulativeNanosForFetchMessage() > 0); } - + public void initializePartitionCheckpointList() { + chkPoints = new TreeMap(); + partitionCheckpointList = new PartitionCheckpointList(chkPoints); + } } diff --git a/messaging-client-databus/src/test/java/com/inmobi/databus/readers/TestCollectorStreamReader.java b/messaging-client-databus/src/test/java/com/inmobi/databus/readers/TestCollectorStreamReader.java index 27f1583..4e5dc50 100644 --- a/messaging-client-databus/src/test/java/com/inmobi/databus/readers/TestCollectorStreamReader.java +++ b/messaging-client-databus/src/test/java/com/inmobi/databus/readers/TestCollectorStreamReader.java @@ -31,9 +31,11 @@ public class TestCollectorStreamReader { private String doesNotExist1 = TestUtil.files[0]; private String doesNotExist2 = TestUtil.files[2]; private String doesNotExist3 = TestUtil.files[7]; + int consumerNumber; @BeforeTest public void setup() throws Exception { + consumerNumber = 1; // setup cluster cluster = TestUtil.setupLocalCluster(this.getClass().getSimpleName(), testStream, partitionId, files, null, 0); @@ -52,7 +54,8 @@ public void cleanup() throws IOException { @Test public void testInitialize() throws Exception { CollectorReaderStatsExposer metrics = new - CollectorReaderStatsExposer(testStream, "c1", partitionId.toString()); + CollectorReaderStatsExposer(testStream, "c1", partitionId.toString(), + consumerNumber); // Read from start cReader = new CollectorStreamReader(partitionId, FileSystem.get( cluster.getHadoopConf()), testStream, @@ -142,7 +145,8 @@ private void readFile(int fileNum, int startIndex) throws Exception { @Test public void testReadFromStart() throws Exception { CollectorReaderStatsExposer metrics = new - CollectorReaderStatsExposer(testStream, "c1", partitionId.toString()); + CollectorReaderStatsExposer(testStream, "c1", partitionId.toString(), + consumerNumber); cReader = new CollectorStreamReader(partitionId, FileSystem.get(cluster.getHadoopConf()), testStream, CollectorStreamReader.getCollectorDir(cluster, testStream, collectorName), @@ -165,7 +169,8 @@ public void testReadFromStart() throws Exception { @Test public void testReadFromCheckpoint() throws Exception { CollectorReaderStatsExposer metrics = new - CollectorReaderStatsExposer(testStream, "c1", partitionId.toString()); + CollectorReaderStatsExposer(testStream, "c1", partitionId.toString(), + consumerNumber); cReader = new CollectorStreamReader(partitionId, FileSystem.get(cluster.getHadoopConf()), testStream, CollectorStreamReader.getCollectorDir(cluster, testStream, collectorName), @@ -188,7 +193,8 @@ public void testReadFromCheckpoint() throws Exception { @Test public void testReadFromTimeStamp() throws Exception { CollectorReaderStatsExposer metrics = new - CollectorReaderStatsExposer(testStream, "c1", partitionId.toString()); + CollectorReaderStatsExposer(testStream, "c1", partitionId.toString(), + consumerNumber); cReader = new CollectorStreamReader(partitionId, FileSystem.get(cluster.getHadoopConf()), testStream, CollectorStreamReader.getCollectorDir(cluster, testStream, collectorName), diff --git a/messaging-client-databus/src/test/java/com/inmobi/databus/readers/TestHadoopStreamReader.java b/messaging-client-databus/src/test/java/com/inmobi/databus/readers/TestHadoopStreamReader.java index c06984f..f73ae23 100644 --- a/messaging-client-databus/src/test/java/com/inmobi/databus/readers/TestHadoopStreamReader.java +++ b/messaging-client-databus/src/test/java/com/inmobi/databus/readers/TestHadoopStreamReader.java @@ -1,6 +1,8 @@ package com.inmobi.databus.readers; import java.io.IOException; +import java.util.TreeMap; +import java.util.TreeSet; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; @@ -10,12 +12,15 @@ import org.testng.annotations.BeforeTest; import org.testng.annotations.Test; +import com.inmobi.databus.partition.PartitionCheckpoint; +import com.inmobi.databus.partition.PartitionCheckpointList; import com.inmobi.messaging.consumer.util.HadoopUtil; public class TestHadoopStreamReader extends TestAbstractDatabusWaitingReader{ @BeforeTest public void setup() throws Exception { + consumerNumber = 1; files = new String[] {HadoopUtil.files[1], HadoopUtil.files[3], HadoopUtil.files[5]}; conf = new Configuration(); @@ -26,6 +31,12 @@ public void setup() throws Exception { HadoopUtil.setupHadoopCluster(conf, files, null, finalFiles, streamDir); inputFormatClass = SequenceFileInputFormat.class.getCanonicalName(); encoded = false; + partitionMinList = new TreeSet(); + for (int i = 0; i < 60; i++) { + partitionMinList.add(i); + } + chkPoints = new TreeMap(); + partitionCheckpointList = new PartitionCheckpointList(chkPoints); } @AfterTest @@ -41,7 +52,7 @@ public void testInitialize() throws Exception { @Test public void testReadFromStart() throws Exception { super.testReadFromStart(); - } + } @Test public void testReadFromCheckpoint() throws Exception { diff --git a/messaging-client-databus/src/test/java/com/inmobi/databus/readers/TestLocalStreamCollectorReader.java b/messaging-client-databus/src/test/java/com/inmobi/databus/readers/TestLocalStreamCollectorReader.java index 84832fb..91b7a01 100644 --- a/messaging-client-databus/src/test/java/com/inmobi/databus/readers/TestLocalStreamCollectorReader.java +++ b/messaging-client-databus/src/test/java/com/inmobi/databus/readers/TestLocalStreamCollectorReader.java @@ -34,10 +34,12 @@ public class TestLocalStreamCollectorReader { private String doesNotExist2 = TestUtil.files[2]; private String doesNotExist3 = TestUtil.files[7]; Configuration conf; + int consumerNumber; @BeforeTest public void setup() throws Exception { // initialize config + consumerNumber = 1; cluster = TestUtil.setupLocalCluster(this.getClass().getSimpleName(), testStream, partitionId, files, null, databusFiles, 3); conf = cluster.getHadoopConf(); @@ -51,7 +53,8 @@ public void cleanup() throws IOException { @Test public void testInitialize() throws Exception { CollectorReaderStatsExposer metrics = new - CollectorReaderStatsExposer(testStream, "c1", partitionId.toString()); + CollectorReaderStatsExposer(testStream, "c1", partitionId.toString(), + consumerNumber); // Read from start lreader = new LocalStreamCollectorReader(partitionId, FileSystem.get(cluster.getHadoopConf()), testStream, @@ -131,7 +134,8 @@ private void readFile(int fileNum, int startIndex) throws Exception { @Test public void testReadFromStart() throws Exception { CollectorReaderStatsExposer metrics = new - CollectorReaderStatsExposer(testStream, "c1", partitionId.toString()); + CollectorReaderStatsExposer(testStream, "c1", partitionId.toString(), + consumerNumber); lreader = new LocalStreamCollectorReader(partitionId, FileSystem.get(cluster.getHadoopConf()), testStream, DatabusStreamReader.getStreamsLocalDir(cluster, testStream), conf, @@ -154,7 +158,8 @@ public void testReadFromStart() throws Exception { @Test public void testReadFromCheckpoint() throws Exception { CollectorReaderStatsExposer metrics = new - CollectorReaderStatsExposer(testStream, "c1", partitionId.toString()); + CollectorReaderStatsExposer(testStream, "c1", partitionId.toString(), + consumerNumber); lreader = new LocalStreamCollectorReader(partitionId, FileSystem.get(cluster.getHadoopConf()), testStream, DatabusStreamReader.getStreamsLocalDir(cluster, testStream), conf, 0L, @@ -179,7 +184,8 @@ public void testReadFromCheckpoint() throws Exception { @Test public void testReadFromTimeStamp() throws Exception { CollectorReaderStatsExposer metrics = new - CollectorReaderStatsExposer(testStream, "c1", partitionId.toString()); + CollectorReaderStatsExposer(testStream, "c1", partitionId.toString(), + consumerNumber); lreader = new LocalStreamCollectorReader(partitionId, FileSystem.get(cluster.getHadoopConf()), testStream, DatabusStreamReader.getStreamsLocalDir(cluster, testStream), conf, 0L, diff --git a/messaging-client-databus/src/test/java/com/inmobi/databus/readers/TestLocalStreamReader.java b/messaging-client-databus/src/test/java/com/inmobi/databus/readers/TestLocalStreamReader.java index 7956d06..867c61f 100644 --- a/messaging-client-databus/src/test/java/com/inmobi/databus/readers/TestLocalStreamReader.java +++ b/messaging-client-databus/src/test/java/com/inmobi/databus/readers/TestLocalStreamReader.java @@ -1,6 +1,8 @@ package com.inmobi.databus.readers; import java.io.IOException; +import java.util.TreeMap; +import java.util.TreeSet; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -10,7 +12,9 @@ import org.testng.annotations.Test; import com.inmobi.databus.Cluster; +import com.inmobi.databus.partition.PartitionCheckpointList; import com.inmobi.databus.partition.PartitionId; +import com.inmobi.databus.partition.PartitionCheckpoint; import com.inmobi.messaging.consumer.util.TestUtil; public class TestLocalStreamReader extends TestAbstractDatabusWaitingReader{ @@ -18,6 +22,7 @@ public class TestLocalStreamReader extends TestAbstractDatabusWaitingReader{ @BeforeTest public void setup() throws Exception { + consumerNumber = 1; // initialize config cluster = TestUtil.setupLocalCluster(this.getClass().getSimpleName(), testStream, new PartitionId(clusterName, collectorName), files, null, @@ -27,6 +32,12 @@ testStream, new PartitionId(clusterName, collectorName), files, null, streamDir = getStreamsDir(); inputFormatClass = TextInputFormat.class.getCanonicalName(); encoded = true; + partitionMinList = new TreeSet(); + for (int i = 0; i < 60; i++) { + partitionMinList.add(i); + } + chkPoints = new TreeMap(); + partitionCheckpointList = new PartitionCheckpointList(chkPoints); } @AfterTest diff --git a/messaging-client-databus/src/test/java/com/inmobi/databus/readers/TestMergeStreamMultipleCollectors.java b/messaging-client-databus/src/test/java/com/inmobi/databus/readers/TestMergeStreamMultipleCollectors.java index 70b1b69..9e84b97 100644 --- a/messaging-client-databus/src/test/java/com/inmobi/databus/readers/TestMergeStreamMultipleCollectors.java +++ b/messaging-client-databus/src/test/java/com/inmobi/databus/readers/TestMergeStreamMultipleCollectors.java @@ -1,6 +1,10 @@ package com.inmobi.databus.readers; import java.io.IOException; +import java.util.Map; +import java.util.Set; +import java.util.TreeMap; +import java.util.TreeSet; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; @@ -12,6 +16,8 @@ import org.testng.annotations.Test; import com.inmobi.databus.Cluster; +import com.inmobi.databus.partition.PartitionCheckpoint; +import com.inmobi.databus.partition.PartitionCheckpointList; import com.inmobi.databus.partition.PartitionId; import com.inmobi.messaging.consumer.util.TestUtil; import com.inmobi.messaging.metrics.PartitionReaderStatsExposer; @@ -31,9 +37,14 @@ public class TestMergeStreamMultipleCollectors { Path[] databusFiles2 = new Path[3]; Configuration conf; boolean encoded = true; - + Set partitionMinList; + PartitionCheckpointList partitionCheckpointList; + Map chkPoints; + int conusmerNumber; + @BeforeTest public void setup() throws Exception { + conusmerNumber = 1; // initialize config cluster = TestUtil.setupLocalCluster(this.getClass().getSimpleName(), testStream, new PartitionId(clusterName, collectors[0]), files, null, @@ -41,6 +52,12 @@ testStream, new PartitionId(clusterName, collectors[0]), files, null, TestUtil.setUpFiles(cluster, collectors[1], files, null, databusFiles2, 0, 3); conf = cluster.getHadoopConf(); + partitionMinList = new TreeSet(); + for (int i = 0; i < 60; i++) { + partitionMinList.add(i); + } + chkPoints = new TreeMap(); + partitionCheckpointList = new PartitionCheckpointList(chkPoints); } @AfterTest @@ -51,12 +68,12 @@ public void cleanup() throws IOException { @Test public void testReadFromStart() throws Exception { PartitionReaderStatsExposer metrics = new PartitionReaderStatsExposer( - testStream, "c1", partitionId.toString()); + testStream, "c1", partitionId.toString(), conusmerNumber); reader = new DatabusStreamWaitingReader(partitionId, FileSystem.get(cluster.getHadoopConf()), DatabusStreamReader.getStreamsDir(cluster, testStream), TextInputFormat.class.getCanonicalName(), - conf, 1000, metrics, false); + conf, 1000, metrics, false, partitionMinList, partitionCheckpointList); reader.build(CollectorStreamReader.getDateFromCollectorFile(files[0])); reader.initFromStart(); Assert.assertNotNull(reader.getCurrentFile()); diff --git a/messaging-client-databus/src/test/java/com/inmobi/databus/readers/TestMergeStreamReader.java b/messaging-client-databus/src/test/java/com/inmobi/databus/readers/TestMergeStreamReader.java index 4da26f1..a871a53 100644 --- a/messaging-client-databus/src/test/java/com/inmobi/databus/readers/TestMergeStreamReader.java +++ b/messaging-client-databus/src/test/java/com/inmobi/databus/readers/TestMergeStreamReader.java @@ -1,18 +1,19 @@ package com.inmobi.databus.readers; import java.io.IOException; +import java.util.TreeMap; +import java.util.TreeSet; -import org.apache.commons.codec.binary.Base64; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.mapred.TextInputFormat; -import org.testng.Assert; import org.testng.annotations.AfterTest; import org.testng.annotations.BeforeTest; import org.testng.annotations.Test; import com.inmobi.databus.Cluster; import com.inmobi.databus.partition.PartitionId; +import com.inmobi.databus.partition.PartitionCheckpoint; import com.inmobi.messaging.consumer.util.TestUtil; public class TestMergeStreamReader extends TestAbstractDatabusWaitingReader{ @@ -21,6 +22,7 @@ public class TestMergeStreamReader extends TestAbstractDatabusWaitingReader{ @BeforeTest public void setup() throws Exception { + consumerNumber = 1; // initialize config cluster = TestUtil.setupLocalCluster(this.getClass().getSimpleName(), testStream, new PartitionId(clusterName, collectorName), files, null, @@ -30,6 +32,11 @@ testStream, new PartitionId(clusterName, collectorName), files, null, streamDir = getStreamsDir(); inputFormatClass = TextInputFormat.class.getCanonicalName(); encoded = true; + partitionMinList = new TreeSet(); + for (int i = 0; i < 60; i++) { + partitionMinList.add(i); + } + chkPoints = new TreeMap(); } @AfterTest diff --git a/messaging-client-databus/src/test/java/com/inmobi/messaging/consumer/databus/TestAbstractDatabusConsumer.java b/messaging-client-databus/src/test/java/com/inmobi/messaging/consumer/databus/TestAbstractDatabusConsumer.java index 33c3b1e..0aefd03 100644 --- a/messaging-client-databus/src/test/java/com/inmobi/messaging/consumer/databus/TestAbstractDatabusConsumer.java +++ b/messaging-client-databus/src/test/java/com/inmobi/messaging/consumer/databus/TestAbstractDatabusConsumer.java @@ -30,7 +30,8 @@ public abstract class TestAbstractDatabusConsumer { public void setup(int numFileToMove) throws Exception { ClientConfig config = loadConfig(); - testConsumer = new DatabusConsumer(); + testConsumer = getConsumerInstance(); + //System.out.println(testConsumer.getClass().getCanonicalName()); testConsumer.initializeConfig(config); // setup stream, collector dirs and data files @@ -67,6 +68,10 @@ public void setup(int numFileToMove) throws Exception { } } + protected DatabusConsumer getConsumerInstance() { + return new DatabusConsumer(); + } + abstract ClientConfig loadConfig(); void assertMessages( diff --git a/messaging-client-databus/src/test/java/com/inmobi/messaging/consumer/databus/TestConsumerPartitionMinList.java b/messaging-client-databus/src/test/java/com/inmobi/messaging/consumer/databus/TestConsumerPartitionMinList.java new file mode 100644 index 0000000..40c2626 --- /dev/null +++ b/messaging-client-databus/src/test/java/com/inmobi/messaging/consumer/databus/TestConsumerPartitionMinList.java @@ -0,0 +1,61 @@ +package com.inmobi.messaging.consumer.databus; + +import java.util.Set; +import java.util.TreeSet; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; + +import org.testng.Assert; +import org.testng.annotations.AfterTest; +import org.testng.annotations.BeforeTest; +import org.testng.annotations.Test; + +import com.inmobi.messaging.ClientConfig; + +public class TestConsumerPartitionMinList { + + protected static final Log LOG = LogFactory.getLog( + TestConsumerPartitionMinList.class); + int consumerId = 1; + int totalNumberOfConsumers = 2; + Set expectedPartitionMinList; + DatabusConsumer testConsumer; + + @BeforeTest + public void setup() throws Exception { + ClientConfig config = ClientConfig.loadFromClasspath( + "messaging-consumer-conf4.properties"); + + expectedPartitionMinList = new TreeSet(); + testConsumer = new DatabusConsumer(); + testConsumer.initializeConfig(config); + if (totalNumberOfConsumers > 0 && consumerId > 0) { + expectedPartitionMinList(); + } + } + + public void expectedPartitionMinList() throws Exception { + for (int i = 0; i < 60; i++ ) { + if ((i % totalNumberOfConsumers) == (consumerId - 1)) { + expectedPartitionMinList.add(i); + } + } + } + + @Test + public void testPartitionMinList() { + Set actualPartitionMinList = testConsumer.getPartitionMinList(); + Assert.assertEquals(consumerId, testConsumer.consumerNumber); + Assert.assertEquals(totalNumberOfConsumers, testConsumer.totalConsumers); + Assert.assertEquals(expectedPartitionMinList.size(), + actualPartitionMinList.size()); + expectedPartitionMinList.containsAll(actualPartitionMinList); + } + + @AfterTest + public void cleanUp() { + testConsumer.close(); + } + +} diff --git a/messaging-client-databus/src/test/java/com/inmobi/messaging/consumer/hadoop/TestAbstractHadoopConsumer.java b/messaging-client-databus/src/test/java/com/inmobi/messaging/consumer/hadoop/TestAbstractHadoopConsumer.java index 17f769e..3eaccde 100644 --- a/messaging-client-databus/src/test/java/com/inmobi/messaging/consumer/hadoop/TestAbstractHadoopConsumer.java +++ b/messaging-client-databus/src/test/java/com/inmobi/messaging/consumer/hadoop/TestAbstractHadoopConsumer.java @@ -71,7 +71,7 @@ public void testMarkAndResetWithStartTime() throws Exception { rootDirs[0].toString()); ConsumerUtil.testMarkAndResetWithStartTime(config, testStream, consumerName, DatabusStreamWaitingReader.getDateFromStreamDir( - rootDirs[0], finalPaths[0][1]), true); + rootDirs[0], finalPaths[0][1]), true); } public void testSuffixDirs() throws Exception { diff --git a/messaging-client-databus/src/test/java/com/inmobi/messaging/consumer/hadoop/TestConsumerPartitionRetention.java b/messaging-client-databus/src/test/java/com/inmobi/messaging/consumer/hadoop/TestConsumerPartitionRetention.java new file mode 100644 index 0000000..09714c0 --- /dev/null +++ b/messaging-client-databus/src/test/java/com/inmobi/messaging/consumer/hadoop/TestConsumerPartitionRetention.java @@ -0,0 +1,123 @@ +package com.inmobi.messaging.consumer.hadoop; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; + +import org.testng.Assert; +import org.testng.annotations.AfterTest; +import org.testng.annotations.BeforeTest; +import org.testng.annotations.Test; + +import com.inmobi.messaging.ClientConfig; +import com.inmobi.messaging.Message; +import com.inmobi.messaging.consumer.util.HadoopUtil; +import com.inmobi.messaging.consumer.util.MessageUtil; +import com.inmobi.messaging.consumer.hadoop.HadoopConsumer; + +public class TestConsumerPartitionRetention { + protected static final Log LOG = LogFactory.getLog( + TestConsumerPartitionRetention.class); + HadoopConsumer consumer; + HadoopConsumer secondConsumer; + String firstConfFile = "messaging-consumer-hadoop-conf7.properties"; + String secondConfFile = "messaging-consumer-hadoop-conf8.properties"; + String streamName = "testStream"; + List firstConsumedMessages; + List secondConsumedMessages; + ClientConfig config; + ClientConfig secondconfig; + int numMessagesPerFile = 100; + int numDataFiles; + int numSuffixDirs; + protected String[] dataFiles = new String[] {HadoopUtil.files[0], + HadoopUtil.files[1], HadoopUtil.files[2], HadoopUtil.files[3]}; + protected String[] suffixDirs; + protected String consumerName; + protected Path[] rootDirs; + Path [][] finalPaths; + Configuration conf; + + @BeforeTest + public void setup() throws Exception { + config = ClientConfig.loadFromClasspath(firstConfFile); + secondconfig = ClientConfig.loadFromClasspath(secondConfFile); + firstConsumedMessages = new ArrayList(); + secondConsumedMessages = new ArrayList(); + createFiles(consumer); + + consumer = new HadoopConsumer(); + secondConsumer = new HadoopConsumer(); + } + + public void createFiles(HadoopConsumer consumer) throws Exception { + consumer = new HadoopConsumer(); + consumer.initializeConfig(config); + conf = consumer.getHadoopConf(); + rootDirs = consumer.getRootDirs(); + LOG.info("number of root dirs "+ rootDirs.length); + numSuffixDirs = suffixDirs != null ? suffixDirs.length : 1; + numDataFiles = dataFiles != null ? dataFiles.length : 1; + finalPaths = new Path[rootDirs.length][numSuffixDirs * numDataFiles]; + for (int i = 0; i < rootDirs.length; i++) { + HadoopUtil.setupHadoopCluster( + conf, dataFiles, suffixDirs, finalPaths[i], rootDirs[i]); + } + HadoopUtil.setUpHadoopFiles(rootDirs[0], conf, + new String[] {"_SUCCESS", "_DONE"}, suffixDirs, null); + } + + @Test + public void testWithRetentionPeriod() throws Exception { + + consumer.init(streamName, consumerName, null, config); + LOG.info("topicname is" + streamName + consumer.getTopicName()); + Assert.assertEquals(consumer.getTopicName(), streamName); + Assert.assertEquals(consumer.getConsumerName(), consumerName); + + secondConsumer.init(streamName, consumerName, null, secondconfig); + Assert.assertEquals(secondConsumer.getTopicName(), streamName); + Assert.assertEquals(secondConsumer.getConsumerName(), consumerName); + LOG.info("checking: whether consumers started at same time" + + consumer.getStartTime() + " " + secondConsumer.getStartTime()); + Assert.assertEquals(secondConsumer.getStartTime(), consumer.getStartTime()); + + //consume all messages + while (firstConsumedMessages.size() < 600) { + Message msg = consumer.next(); + firstConsumedMessages.add(getMessage(msg.getData().array())); + } + consumer.close(); + LOG.info("number of msgs consumed by first consumer" + + firstConsumedMessages.size()); + + while (secondConsumedMessages.size() < 600) { + Message msgs = secondConsumer.next(); + secondConsumedMessages.add(getMessage(msgs.getData().array())); + } + secondConsumer.close(); + LOG.info("number of messages consumed messages by second consumer " + + secondConsumedMessages.size()); + + Assert.assertEquals(firstConsumedMessages.size() + + secondConsumedMessages.size(), 1200); + } + + @AfterTest + public void cleanup() throws Exception { + FileSystem lfs = FileSystem.getLocal(conf); + for (Path rootDir : rootDirs) { + lfs.delete(rootDir.getParent(), true); + } + } + + private static String getMessage(byte[] array) throws IOException { + return MessageUtil.getTextMessage(array).toString(); + } +} diff --git a/messaging-client-databus/src/test/java/com/inmobi/messaging/consumer/hadoop/TestConsumerPartitionStartTime.java b/messaging-client-databus/src/test/java/com/inmobi/messaging/consumer/hadoop/TestConsumerPartitionStartTime.java new file mode 100644 index 0000000..4a343d8 --- /dev/null +++ b/messaging-client-databus/src/test/java/com/inmobi/messaging/consumer/hadoop/TestConsumerPartitionStartTime.java @@ -0,0 +1,131 @@ +package com.inmobi.messaging.consumer.hadoop; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Calendar; +import java.util.Date; +import java.util.List; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; + +import org.testng.Assert; +import org.testng.annotations.AfterTest; +import org.testng.annotations.BeforeTest; +import org.testng.annotations.Test; + +import com.inmobi.messaging.ClientConfig; +import com.inmobi.messaging.Message; +import com.inmobi.messaging.consumer.util.HadoopUtil; +import com.inmobi.messaging.consumer.util.MessageUtil; +import com.inmobi.messaging.consumer.hadoop.HadoopConsumer; + +public class TestConsumerPartitionStartTime { + + protected static final Log LOG = LogFactory.getLog( + TestConsumerPartitionStartTime.class); + HadoopConsumer consumer; + HadoopConsumer secondConsumer; + String firstConfFile = "messaging-consumer-hadoop-conf5.properties"; + String secondConfFile = "messaging-consumer-hadoop-conf6.properties"; + String streamName = "testStream"; + Date startTime; + List firstConsumedMessages; + List secondConsumedMessages; + ClientConfig config; + ClientConfig secondconfig; + int numMessagesPerFile = 100; + int numDataFiles; + int numSuffixDirs; + protected String[] dataFiles = new String[] {HadoopUtil.files[0], + HadoopUtil.files[1], + HadoopUtil.files[2], + HadoopUtil.files[3]}; + protected String[] suffixDirs; + protected String consumerName; + protected Path[] rootDirs; + Path [][] finalPaths; + Configuration conf; + + @BeforeTest + public void setup() throws Exception { + Calendar cal = Calendar.getInstance(); + startTime = cal.getTime(); + LOG.info("current time is " + startTime.getMinutes()); + cal.add(cal.MINUTE, -(20)); + startTime = cal.getTime(); + config = ClientConfig.loadFromClasspath(firstConfFile); + secondconfig = ClientConfig.loadFromClasspath(secondConfFile); + firstConsumedMessages = new ArrayList(); + secondConsumedMessages = new ArrayList(); + createFiles(consumer); + + consumer = new HadoopConsumer(); + secondConsumer = new HadoopConsumer(); + } + + public void createFiles(HadoopConsumer consumer) throws Exception { + consumer = new HadoopConsumer(); + consumer.initializeConfig(config); + + conf = consumer.getHadoopConf(); + Assert.assertEquals(conf.get("myhadoop.property"), "myvalue"); + + rootDirs = consumer.getRootDirs(); + LOG.info("number of root dirs "+ rootDirs.length); + numSuffixDirs = suffixDirs != null ? suffixDirs.length : 1; + numDataFiles = dataFiles != null ? dataFiles.length : 1; + finalPaths = new Path[rootDirs.length][numSuffixDirs * numDataFiles]; + for (int i = 0; i < rootDirs.length; i++) { + HadoopUtil.setupHadoopCluster( + conf, dataFiles, suffixDirs, finalPaths[i], rootDirs[i]); + } + HadoopUtil.setUpHadoopFiles(rootDirs[0], conf, + new String[] {"_SUCCESS", "_DONE"}, suffixDirs, null); + } + + @Test + public void testWithStartTime() throws Exception { + consumer.init(streamName, consumerName, startTime, config); + Assert.assertEquals(consumer.getTopicName(), streamName); + Assert.assertEquals(consumer.getConsumerName(), consumerName); + + secondConsumer.init(streamName, consumerName, startTime, secondconfig); + Assert.assertEquals(secondConsumer.getTopicName(), streamName); + Assert.assertEquals(secondConsumer.getConsumerName(), consumerName); + Assert.assertEquals(secondConsumer.getStartTime(), consumer.getStartTime()); + + //consume all messages + while (firstConsumedMessages.size() < 600) { + Message msg = consumer.next(); + firstConsumedMessages.add(getMessage(msg.getData().array())); + } + consumer.close(); + LOG.info("msgs consumed by first consumer" + firstConsumedMessages.size()); + + while (secondConsumedMessages.size() < 600) { + Message msgs = secondConsumer.next(); + secondConsumedMessages.add(getMessage(msgs.getData().array())); + } + secondConsumer.close(); + LOG.info("msgs consumed by second consumer " + secondConsumedMessages.size()); + + Assert.assertEquals(firstConsumedMessages.size() + + secondConsumedMessages.size(), 1200); + } + + @AfterTest + public void cleanup() throws Exception { + FileSystem lfs = FileSystem.getLocal(conf); + for (Path rootDir : rootDirs) { + lfs.delete(rootDir.getParent(), true); + } + } + + private static String getMessage(byte[] array) throws IOException { + return MessageUtil.getTextMessage(array).toString(); + } +} diff --git a/messaging-client-databus/src/test/java/com/inmobi/messaging/consumer/hadoop/TestHadoopConsumer.java b/messaging-client-databus/src/test/java/com/inmobi/messaging/consumer/hadoop/TestHadoopConsumer.java index d2ea6d5..91f5939 100644 --- a/messaging-client-databus/src/test/java/com/inmobi/messaging/consumer/hadoop/TestHadoopConsumer.java +++ b/messaging-client-databus/src/test/java/com/inmobi/messaging/consumer/hadoop/TestHadoopConsumer.java @@ -32,7 +32,7 @@ public void setup() throws Exception { public void testMarkAndReset() throws Exception { super.testMarkAndReset(); } - + @Test public void testMarkAndResetWithStartTime() throws Exception { super.testMarkAndResetWithStartTime(); @@ -47,7 +47,7 @@ public void testMultipleClusters() throws Exception { public void testMultipleClusters2() throws Exception { super.testMultipleClusters2(); } - + @AfterTest public void cleanup() throws IOException { super.cleanup(); diff --git a/messaging-client-databus/src/test/java/com/inmobi/messaging/consumer/hadoop/TestHadoopConsumerWithPartitionList.java b/messaging-client-databus/src/test/java/com/inmobi/messaging/consumer/hadoop/TestHadoopConsumerWithPartitionList.java new file mode 100644 index 0000000..f123c78 --- /dev/null +++ b/messaging-client-databus/src/test/java/com/inmobi/messaging/consumer/hadoop/TestHadoopConsumerWithPartitionList.java @@ -0,0 +1,107 @@ +package com.inmobi.messaging.consumer.hadoop; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Date; +import java.util.List; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.testng.Assert; +import org.testng.annotations.AfterTest; +import org.testng.annotations.BeforeTest; +import org.testng.annotations.Test; + +import com.inmobi.databus.readers.DatabusStreamWaitingReader; +import com.inmobi.messaging.ClientConfig; +import com.inmobi.messaging.Message; +import com.inmobi.messaging.consumer.BaseMessageConsumerStatsExposer; +import com.inmobi.messaging.consumer.databus.AbstractMessagingDatabusConsumer; +import com.inmobi.messaging.consumer.util.ConsumerUtil; +import com.inmobi.messaging.consumer.util.HadoopUtil; +import com.inmobi.messaging.consumer.util.MessageUtil; + +public class TestHadoopConsumerWithPartitionList { + protected static final Log LOG = LogFactory.getLog( + TestHadoopConsumerWithPartitionList.class); + HadoopConsumer consumer; + HadoopConsumer secondConsumer; + String firstConfFile = "messaging-consumer-hadoop-conf11.properties"; + String secondConfFile = "messaging-consumer-hadoop-conf12.properties"; + String streamName = "testStream"; + int numMessagesPerFile = 100; + int numDataFiles; + int numSuffixDirs; + protected String[] dataFiles = new String[] {HadoopUtil.files[0], + HadoopUtil.files[1], + HadoopUtil.files[2], HadoopUtil.files[3]}; + protected String[] suffixDirs; + protected String consumerName = "c1"; + protected Path[] rootDirs; + Path [][] finalPaths; + Configuration conf; + ClientConfig firstConsumerConfig; + ClientConfig secondConsuemrConfig; + protected String ck8; + + boolean hadoop = true; + + @BeforeTest + public void setup() throws Exception { + firstConsumerConfig = ClientConfig.loadFromClasspath(firstConfFile); + secondConsuemrConfig = ClientConfig.loadFromClasspath(secondConfFile); + + createFiles(consumer); + + ck8 = "/tmp/test/hadoop/8/checkpoint"; + consumer = new HadoopConsumer(); + secondConsumer = new HadoopConsumer(); + } + + public void createFiles(HadoopConsumer consumer) throws Exception { + consumer = new HadoopConsumer(); + consumer.initializeConfig(firstConsumerConfig); + + conf = consumer.getHadoopConf(); + Assert.assertEquals(conf.get("myhadoop.property"), "myvalue"); + + rootDirs = consumer.getRootDirs(); + LOG.info("size is" + rootDirs.length); + LOG.info("number of root dirs "+ rootDirs.length); + numSuffixDirs = suffixDirs != null ? suffixDirs.length : 1; + numDataFiles = dataFiles != null ? dataFiles.length : 1; + finalPaths = new Path[rootDirs.length][numSuffixDirs * numDataFiles]; + for (int i = 0; i < rootDirs.length; i++) { + HadoopUtil.setupHadoopCluster( + conf, dataFiles, suffixDirs, finalPaths[i], rootDirs[i]); + } + HadoopUtil.setUpHadoopFiles(rootDirs[0], conf, + new String[] {"_SUCCESS", "_DONE"}, suffixDirs, null); + } + + @Test + public void testConsumerMarkAndResetWithStartTime() throws Exception { + firstConsumerConfig.set(HadoopConsumerConfig.rootDirsConfig, + rootDirs[1].toString()); + firstConsumerConfig.set(HadoopConsumerConfig.checkpointDirConfig, + ck8); + secondConsuemrConfig.set(HadoopConsumerConfig.rootDirsConfig, + rootDirs[1].toString()); + secondConsuemrConfig.set(HadoopConsumerConfig.checkpointDirConfig, ck8); + ConsumerUtil.testConsumerMarkAndResetWithStartTime(firstConsumerConfig, + secondConsuemrConfig, streamName, consumerName, + DatabusStreamWaitingReader.getDateFromStreamDir( + rootDirs[1], finalPaths[0][1]), true); + } + + @AfterTest + public void cleanup() throws IOException { + FileSystem lfs = FileSystem.getLocal(conf); + for (Path rootDir : rootDirs) { + lfs.delete(rootDir.getParent(), true); + } + } +} \ No newline at end of file diff --git a/messaging-client-databus/src/test/java/com/inmobi/messaging/consumer/util/ConsumerUtil.java b/messaging-client-databus/src/test/java/com/inmobi/messaging/consumer/util/ConsumerUtil.java index 2f0d31c..ea8a805 100644 --- a/messaging-client-databus/src/test/java/com/inmobi/messaging/consumer/util/ConsumerUtil.java +++ b/messaging-client-databus/src/test/java/com/inmobi/messaging/consumer/util/ConsumerUtil.java @@ -2,7 +2,10 @@ import java.io.IOException; import java.util.Date; +import java.util.Map; +import java.util.TreeMap; +import org.mortbay.log.Log; import org.testng.Assert; import com.inmobi.messaging.ClientConfig; @@ -10,6 +13,8 @@ import com.inmobi.messaging.consumer.BaseMessageConsumerStatsExposer; import com.inmobi.messaging.consumer.databus.AbstractMessagingDatabusConsumer; import com.inmobi.messaging.consumer.databus.Checkpoint; +import com.inmobi.messaging.consumer.databus.CheckpointList; +import com.inmobi.messaging.consumer.databus.ConsumerCheckpoint; import com.inmobi.messaging.consumer.databus.DatabusConsumer; import com.inmobi.messaging.consumer.hadoop.HadoopConsumer; @@ -22,6 +27,7 @@ public static void assertMessages(ClientConfig config, String streamName, int numCounters = numClusters * numCollectors; int totalMessages = numCounters * numDataFiles * numMessagesPerFile; int[] counter = new int[numCounters]; + int count = 0; for (int i = 0; i checkpointMap = new TreeMap(); + if(temp instanceof CheckpointList) { + //Do a deep copy of the Tree Map, as the entry sets in original map can change + for(Map.Entry entry: ((CheckpointList) temp). + getCheckpoints().entrySet()) { + checkpointMap.put(entry.getKey(), new Checkpoint(entry.getValue().toBytes())); + } + } else { + lastCheckpoint = new Checkpoint( + ((Checkpoint)consumer.getCurrentCheckpoint()).toBytes()); + } + for (int i = 0; i < numCounters; i++) { markedcounter1[i] = counter[i]; @@ -65,11 +84,11 @@ public static void assertMessages(ClientConfig config, String streamName, for (int i= 0; i < numCounters; i++) { Assert.assertEquals(counter[i], numDataFiles * numMessagesPerFile); } - + consumer.reset(); - + for (int i = 0; i < totalMessages/2; i++) { - Message msg = consumer.next(); + Message msg = consumer.next(); String msgStr = getMessage(msg.getData().array(), hadoop); for (int m = 0; m < numCounters; m++) { if (msgStr.equals(MessageUtil.constructMessage(markedcounter1[m]))) { @@ -78,7 +97,7 @@ public static void assertMessages(ClientConfig config, String streamName, } } } - + for (int i= 0; i < numCounters; i++) { Assert.assertEquals(markedcounter1[i], numDataFiles * numMessagesPerFile); } @@ -95,7 +114,13 @@ public static void assertMessages(ClientConfig config, String streamName, consumer = createConsumer(hadoop); consumer.init(streamName, consumerName, null, config); - Assert.assertEquals(consumer.getCurrentCheckpoint(), lastCheckpoint); + + if(temp instanceof CheckpointList) { + Assert.assertEquals(((CheckpointList)consumer.getCurrentCheckpoint()). + getCheckpoints(), checkpointMap); + } else { + Assert.assertEquals(consumer.getCurrentCheckpoint(), lastCheckpoint); + } for (int i = 0; i < totalMessages/2; i++) { Message msg = consumer.next(); String msgStr = getMessage(msg.getData().array(), hadoop); @@ -145,9 +170,10 @@ public static void testMarkAndResetWithStartTime(ClientConfig config, startTime, config); Assert.assertEquals(consumer.getTopicName(), streamName); Assert.assertEquals(consumer.getConsumerName(), consumerName); - + + int i; - for (i = 100; i < 120; i++) { + for (i = 100; i < 120; i++) { Message msg = consumer.next(); Assert.assertEquals(getMessage(msg.getData().array(), hadoop), MessageUtil.constructMessage(i)); @@ -164,12 +190,28 @@ public static void testMarkAndResetWithStartTime(ClientConfig config, for (i = 120; i < 240; i++) { Message msg = consumer.next(); Assert.assertEquals(getMessage(msg.getData().array(), hadoop), - MessageUtil.constructMessage(i)); + MessageUtil.constructMessage(i)); } consumer.mark(); - Checkpoint lastCheckpoint = new Checkpoint( - consumer.getCurrentCheckpoint().toBytes()); + + Checkpoint lastCheckpoint = null; + ConsumerCheckpoint temp = consumer.getCurrentCheckpoint(); + // + Map checkpointMap = new TreeMap(); + if(temp instanceof CheckpointList) { + //Do a deep copy of the Tree Map, as the entry sets in original map can change + for(Map.Entry entry: ((CheckpointList) temp). + getCheckpoints().entrySet()) { + checkpointMap.put(entry.getKey(), new Checkpoint(entry.getValue(). + toBytes())); + } + } + else { + lastCheckpoint = new Checkpoint( + ((Checkpoint)consumer.getCurrentCheckpoint()).toBytes()); + } + for (i = 240; i < 260; i++) { Message msg = consumer.next(); @@ -195,8 +237,12 @@ public static void testMarkAndResetWithStartTime(ClientConfig config, // test checkpoint and consumer crash consumer = createConsumer(hadoop); consumer.init(streamName, consumerName, null, config); - Assert.assertEquals(consumer.getCurrentCheckpoint(), lastCheckpoint); - + if(temp instanceof CheckpointList) { + Assert.assertEquals(((CheckpointList)consumer.getCurrentCheckpoint()). + getCheckpoints(), checkpointMap); + } else { + Assert.assertEquals(consumer.getCurrentCheckpoint(), lastCheckpoint); + } for (i = 240; i < 300; i++) { Message msg = consumer.next(); Assert.assertEquals(getMessage(msg.getData().array(), hadoop), @@ -210,7 +256,7 @@ public static void testMarkAndResetWithStartTime(ClientConfig config, Assert.assertEquals(((BaseMessageConsumerStatsExposer)( consumer.getMetrics())).getNumResetCalls(), 0); Assert.assertEquals(((BaseMessageConsumerStatsExposer)( - consumer.getMetrics())).getNumMessagesConsumed(), 60); + consumer.getMetrics())).getNumMessagesConsumed(), 60); } public static void testMarkAndReset(ClientConfig config, String streamName, @@ -244,20 +290,32 @@ public static void testMarkAndReset(ClientConfig config, String streamName, } consumer.mark(); - Checkpoint lastCheckpoint = new Checkpoint( - consumer.getCurrentCheckpoint().toBytes()); - + Checkpoint lastCheckpoint = null; + ConsumerCheckpoint temp = consumer.getCurrentCheckpoint(); + // + Map checkpointMap = new TreeMap(); + if(temp instanceof CheckpointList) { + //Do a deep copy of the Tree Map, as the entry sets in original map can change + for(Map.Entry entry: ((CheckpointList) temp). + getCheckpoints().entrySet()) { + checkpointMap.put(entry.getKey(), new Checkpoint(entry.getValue(). + toBytes())); + } + } else { + lastCheckpoint = new Checkpoint( + ((Checkpoint)consumer.getCurrentCheckpoint()).toBytes()); + } for (i = 140; i < 160; i++) { - Message msg = consumer.next(); - Assert.assertEquals(getMessage(msg.getData().array(), hadoop), - MessageUtil.constructMessage(i)); + Message msg = consumer.next(); + Assert.assertEquals(getMessage(msg.getData().array(), hadoop), + MessageUtil.constructMessage(i)); } consumer.reset(); for (i = 140; i < 300; i++) { - Message msg = consumer.next(); - Assert.assertEquals(getMessage(msg.getData().array(), hadoop), - MessageUtil.constructMessage(i)); + Message msg = consumer.next(); + Assert.assertEquals(getMessage(msg.getData().array(), hadoop), + MessageUtil.constructMessage(i)); } consumer.close(); @@ -271,8 +329,12 @@ public static void testMarkAndReset(ClientConfig config, String streamName, // test checkpoint and consumer crash consumer = createConsumer(hadoop); consumer.init(streamName, consumerName, null, config); - Assert.assertEquals(consumer.getCurrentCheckpoint(), lastCheckpoint); - + if(temp instanceof CheckpointList) { + Assert.assertEquals(((CheckpointList)consumer.getCurrentCheckpoint()). + getCheckpoints(), checkpointMap); + } else { + Assert.assertEquals(consumer.getCurrentCheckpoint(), lastCheckpoint); + } for (i = 140; i < 300; i++) { Message msg = consumer.next(); Assert.assertEquals(getMessage(msg.getData().array(), hadoop), @@ -287,7 +349,113 @@ public static void testMarkAndReset(ClientConfig config, String streamName, consumer.getMetrics())).getNumResetCalls(), 0); Assert.assertEquals(((BaseMessageConsumerStatsExposer)( consumer.getMetrics())).getNumMessagesConsumed(), 160); - + } + + public static void testConsumerMarkAndResetWithStartTime(ClientConfig config, + ClientConfig secondConfig, String streamName, String consumerName, + Date startTime, boolean hadoop) + throws Exception { + AbstractMessagingDatabusConsumer consumer = createConsumer(hadoop); + AbstractMessagingDatabusConsumer secondConsumer = createConsumer(hadoop); + //first consumer initialization + consumer.init(streamName, consumerName, startTime, config); + //second consumer initialization + secondConsumer.init(streamName, consumerName, startTime, secondConfig); + + Assert.assertEquals(consumer.getTopicName(), streamName); + Assert.assertEquals(consumer.getConsumerName(), consumerName); + Assert.assertEquals(consumer.getStartTime(), secondConsumer.getStartTime()); + + int i; + for (i = 0; i < 5; i++) { + Message msg = secondConsumer.next(); +// Assert.assertEquals(getMessage(msg.getData().array(), hadoop), +// MessageUtil.constructMessage(i)); + } + secondConsumer.mark(); + for (i = 0; i < 25; i++) { + Message msg = consumer.next(); +// Assert.assertEquals(getMessage(msg.getData().array(), hadoop), +// MessageUtil.constructMessage(i)); + } + consumer.mark(); + for (i = 5; i < 10; i++) { + Message msg = secondConsumer.next(); +// Assert.assertEquals(getMessage(msg.getData().array(), hadoop), +// MessageUtil.constructMessage(i)); + } + secondConsumer.reset(); + + for (i = 5; i < 10; i++) { + Message msg = secondConsumer.next(); +// Assert.assertEquals(getMessage(msg.getData().array(), hadoop), +// MessageUtil.constructMessage(i)); + } + secondConsumer.mark(); + + for (i = 25; i < 75; i++) { + Message msg = consumer.next(); +// Assert.assertEquals(getMessage(msg.getData().array(), hadoop), +// MessageUtil.constructMessage(i)); + } + consumer.mark(); + + for (i = 75; i < 80; i++) { + Message msg = consumer.next(); +// Assert.assertEquals(getMessage(msg.getData().array(), hadoop), +// MessageUtil.constructMessage(i)); + } + + consumer.reset(); + + for (i = 75; i < 80; i++) { + Message msg = consumer.next(); +// Assert.assertEquals(getMessage(msg.getData().array(), hadoop), +// MessageUtil.constructMessage(i)); + } + consumer.mark(); + Assert.assertEquals(((BaseMessageConsumerStatsExposer)( + consumer.getMetrics())).getNumMessagesConsumed(), 85); + for (i = 80; i < 82; i++) { + Message msg = consumer.next(); +// Assert.assertEquals(getMessage(msg.getData().array(), hadoop), +// MessageUtil.constructMessage(i)); + } + + consumer.reset(); + for (i = 80; i < 82; i++) { + Message msg = consumer.next(); +// Assert.assertEquals(getMessage(msg.getData().array(), hadoop), +// MessageUtil.constructMessage(i)); + } + consumer.mark(); + consumer.close(); + secondConsumer.close(); + + ConsumerCheckpoint temp = consumer.getCurrentCheckpoint(); + //test checkpoint + Map checkpointMap = new TreeMap(); + if(temp instanceof CheckpointList) { + //Do a deep copy of the Tree Map, as the entry sets in original map can change + for(Map.Entry entry: ((CheckpointList) temp). + getCheckpoints().entrySet()) { + checkpointMap.put(entry.getKey(), new Checkpoint(entry.getValue(). + toBytes())); + } + } + Assert.assertEquals(((CheckpointList)consumer.getCurrentCheckpoint()). + getCheckpoints(), checkpointMap); + Assert.assertEquals(((BaseMessageConsumerStatsExposer)( + consumer.getMetrics())).getNumMarkCalls(), 4); + Assert.assertEquals(((BaseMessageConsumerStatsExposer)( + consumer.getMetrics())).getNumResetCalls(), 2); + Assert.assertEquals(((BaseMessageConsumerStatsExposer)( + secondConsumer.getMetrics())).getNumMarkCalls(), 2); + Assert.assertEquals(((BaseMessageConsumerStatsExposer)( + secondConsumer.getMetrics())).getNumResetCalls(), 1); + Assert.assertEquals(((BaseMessageConsumerStatsExposer)( + consumer.getMetrics())).getNumMessagesConsumed(),89); + } } diff --git a/messaging-client-databus/src/test/java/com/inmobi/messaging/consumer/util/TestUtil.java b/messaging-client-databus/src/test/java/com/inmobi/messaging/consumer/util/TestUtil.java index c18f2af..543529d 100644 --- a/messaging-client-databus/src/test/java/com/inmobi/messaging/consumer/util/TestUtil.java +++ b/messaging-client-databus/src/test/java/com/inmobi/messaging/consumer/util/TestUtil.java @@ -142,8 +142,8 @@ public static void assertBuffer(StreamFile file, int fileNum, int startIndex, for (int i = startIndex; i < (startIndex + numMsgs); i++) { QueueEntry entry = buffer.take(); Assert.assertEquals(entry.getPartitionId(), pid); - Assert.assertEquals(entry.getPartitionChkpoint(), - new PartitionCheckpoint(file, i + 1)); + //Assert.assertEquals(entry.getMessageChkpoint(), // + // new PartitionCheckpoint(file, i + 1)); if (isDatabusData) { Assert.assertEquals(new String(entry.getMessage().getData().array()), MessageUtil.constructMessage(fileIndex + i)); @@ -227,7 +227,7 @@ private static Cluster setupCluster(String className, String testStream, fs.delete(new Path(cluster.getLocalFinalDestDirRoot()), true); fs.delete(new Path(cluster.getFinalDestDirRoot()), true); fs.mkdirs(collectorDir); - + LOG.debug("ready to set up files"); setUpFiles(cluster, pid.getCollector(), collectorFiles, emptyFiles, databusFiles, numFilesToMoveToStreamLocal, numFilesToMoveToStreams); diff --git a/messaging-client-databus/src/test/resources/messaging-consumer-conf.properties b/messaging-client-databus/src/test/resources/messaging-consumer-conf.properties index a97a6b1..db093c1 100644 --- a/messaging-client-databus/src/test/resources/messaging-consumer-conf.properties +++ b/messaging-client-databus/src/test/resources/messaging-consumer-conf.properties @@ -4,3 +4,4 @@ consumer.name=c1 messaging.consumer.checkpoint.dir=file:///tmp/test/databus/checkpoint messaging.consumer.buffer.size=100 databus.consumer.rootdirs=file:///tmp/test/databustest1,file:///tmp/test/databustest2,file:///tmp/test/databustest3 + diff --git a/messaging-client-databus/src/test/resources/messaging-consumer-conf2.properties b/messaging-client-databus/src/test/resources/messaging-consumer-conf2.properties index fefaf77..b745d3e 100644 --- a/messaging-client-databus/src/test/resources/messaging-consumer-conf2.properties +++ b/messaging-client-databus/src/test/resources/messaging-consumer-conf2.properties @@ -4,3 +4,4 @@ consumer.name=c2 messaging.consumer.checkpoint.dir=file:///tmp/test/databus/checkpoint messaging.consumer.buffer.size=100 databus.consumer.rootdirs=file:///tmp/test/databustest4,file:///tmp/test/databustest5,file:///tmp/test/databustest6 + diff --git a/messaging-client-databus/src/test/resources/messaging-consumer-conf20.properties b/messaging-client-databus/src/test/resources/messaging-consumer-conf20.properties new file mode 100644 index 0000000..ac8ed41 --- /dev/null +++ b/messaging-client-databus/src/test/resources/messaging-consumer-conf20.properties @@ -0,0 +1,8 @@ +consumer.classname=com.inmobi.messaging.consumer.databus.DatabusConsumer +topic.name=testclient +consumer.name=c1 +messaging.consumer.checkpoint.dir=/tmp/test/hadoop/1/checkpoint1 +messaging.consumer.buffer.size=100 +databus.consumer.rootdirs=file:///tmp/test/hadoop/1/test1,file:///tmp/test/hadoop/2/test2,file:///tmp/test/hadoop/3/test3 +messaging.consumer.group.membership=1/1 + diff --git a/messaging-client-databus/src/test/resources/messaging-consumer-conf4.properties b/messaging-client-databus/src/test/resources/messaging-consumer-conf4.properties new file mode 100644 index 0000000..29d79de --- /dev/null +++ b/messaging-client-databus/src/test/resources/messaging-consumer-conf4.properties @@ -0,0 +1,8 @@ +consumer.classname=com.inmobi.messaging.consumer.databus.DatabusConsumer +topic.name=testclient +consumer.name=c1 +messaging.consumer.checkpoint.dir=file:///tmp/test/databus/checkpoint +messaging.consumer.buffer.size=100 +databus.consumer.rootdirs=file:///tmp/test/databustest1,file:///tmp/test/databustest2,file:///tmp/test/databustest3 +messaging.consumer.group.membership=1/2 + diff --git a/messaging-client-databus/src/test/resources/messaging-consumer-hadoop-conf.properties b/messaging-client-databus/src/test/resources/messaging-consumer-hadoop-conf.properties index 7ada2e6..7c4fcf3 100644 --- a/messaging-client-databus/src/test/resources/messaging-consumer-hadoop-conf.properties +++ b/messaging-client-databus/src/test/resources/messaging-consumer-hadoop-conf.properties @@ -1,7 +1,7 @@ consumer.classname=com.inmobi.messaging.consumer.hadoop.HadoopConsumer topic.name=testclient consumer.name=c1 -messaging.consumer.checkpoint.dir=/tmp/test/hadoop/checkpoint +messaging.consumer.checkpoint.dir=/tmp/test/hadoop/1/checkpoint2 messaging.consumer.buffer.size=100 messaging.consumer.inputformat.classname=org.apache.hadoop.mapred.SequenceFileInputFormat messaging.consumer.topic.retention.inhours=4 diff --git a/messaging-client-databus/src/test/resources/messaging-consumer-hadoop-conf10.properties b/messaging-client-databus/src/test/resources/messaging-consumer-hadoop-conf10.properties new file mode 100644 index 0000000..77332e9 --- /dev/null +++ b/messaging-client-databus/src/test/resources/messaging-consumer-hadoop-conf10.properties @@ -0,0 +1,10 @@ +consumer.classname=com.inmobi.messaging.consumer.hadoop.HadoopConsumer +topic.name=testclient +consumer.name=c1 +messaging.consumer.checkpoint.dir=/tmp/test/hadoop/1/checkpoint2 +messaging.consumer.buffer.size=100 +messaging.consumer.inputformat.classname=org.apache.hadoop.mapred.SequenceFileInputFormat +messaging.consumer.topic.retention.inhours=4 +messaging.consumer.hadoop.conf=hadoop-conf.xml +hadoop.consumer.rootdirs=file:///tmp/test/hadoop/1/test1 +messaging.consumer.group.membership=1/1 diff --git a/messaging-client-databus/src/test/resources/messaging-consumer-hadoop-conf11.properties b/messaging-client-databus/src/test/resources/messaging-consumer-hadoop-conf11.properties new file mode 100644 index 0000000..ec779d3 --- /dev/null +++ b/messaging-client-databus/src/test/resources/messaging-consumer-hadoop-conf11.properties @@ -0,0 +1,10 @@ +consumer.classname=com.inmobi.messaging.consumer.hadoop.HadoopConsumer +topic.name=testclient +consumer.name=c1 +messaging.consumer.checkpoint.dir=/tmp/test/hadoop/TestConsumerWithPartitionList/checkpoint +messaging.consumer.buffer.size=100 +messaging.consumer.inputformat.classname=org.apache.hadoop.mapred.SequenceFileInputFormat +messaging.consumer.topic.retention.inhours=4 +messaging.consumer.hadoop.conf=hadoop-conf.xml +hadoop.consumer.rootdirs=file:///tmp/test/hadoop/TestConsumerWithPartitionList/1/testclient,file:///tmp/test/hadoop/TestConsumerWithPartitionList/2/testclient,file:///tmp/test/hadoop/TestConsumerWithPartitionList/3/testclient +messaging.consumer.group.membership=1/2 diff --git a/messaging-client-databus/src/test/resources/messaging-consumer-hadoop-conf12.properties b/messaging-client-databus/src/test/resources/messaging-consumer-hadoop-conf12.properties new file mode 100644 index 0000000..92a5a42 --- /dev/null +++ b/messaging-client-databus/src/test/resources/messaging-consumer-hadoop-conf12.properties @@ -0,0 +1,10 @@ +consumer.classname=com.inmobi.messaging.consumer.hadoop.HadoopConsumer +topic.name=testclient +consumer.name=c1 +messaging.consumer.checkpoint.dir=/tmp/test/hadoop/TestConsumerWithPartitionList/checkpoint +messaging.consumer.buffer.size=100 +messaging.consumer.inputformat.classname=org.apache.hadoop.mapred.SequenceFileInputFormat +messaging.consumer.topic.retention.inhours=4 +messaging.consumer.hadoop.conf=hadoop-conf.xml +hadoop.consumer.rootdirs=file:///tmp/test/hadoop/TestConsumerWithPartitionList/1/testclient,file:///tmp/test/hadoop/TestConsumerWithPartitionList/2/testclient,file:///tmp/test/hadoop/TestConsumerWithPartitionList/3/testclient +messaging.consumer.group.membership=2/2 diff --git a/messaging-client-databus/src/test/resources/messaging-consumer-hadoop-conf2.properties b/messaging-client-databus/src/test/resources/messaging-consumer-hadoop-conf2.properties index 2f1ff29..491ddf3 100644 --- a/messaging-client-databus/src/test/resources/messaging-consumer-hadoop-conf2.properties +++ b/messaging-client-databus/src/test/resources/messaging-consumer-hadoop-conf2.properties @@ -6,4 +6,4 @@ messaging.consumer.buffer.size=100 messaging.consumer.inputformat.classname=org.apache.hadoop.mapred.SequenceFileInputFormat messaging.consumer.topic.retention.inhours=4 messaging.consumer.hadoop.conf=hadoop-conf.xml -hadoop.consumer.rootdirs=file:///tmp/test/hadoop/4/test4,file:///tmp/test/hadoop/5/test5,file:///tmp/test/hadoop/6/test6 +hadoop.consumer.rootdirs=file:///tmp/test/hadoop/11/test1,file:///tmp/test/hadoop/12/test2,file:///tmp/test/hadoop/13/test3 diff --git a/messaging-client-databus/src/test/resources/messaging-consumer-hadoop-conf5.properties b/messaging-client-databus/src/test/resources/messaging-consumer-hadoop-conf5.properties new file mode 100644 index 0000000..df52b9b --- /dev/null +++ b/messaging-client-databus/src/test/resources/messaging-consumer-hadoop-conf5.properties @@ -0,0 +1,10 @@ +consumer.classname=com.inmobi.messaging.consumer.hadoop.HadoopConsumer +topic.name=testclient +consumer.name=c1 +messaging.consumer.checkpoint.dir=/tmp/test/hadoop/TestConsumerPartitionStartTime/checkpoint +messaging.consumer.buffer.size=100 +messaging.consumer.inputformat.classname=org.apache.hadoop.mapred.SequenceFileInputFormat +messaging.consumer.topic.retention.inhours=4 +messaging.consumer.hadoop.conf=hadoop-conf.xml +hadoop.consumer.rootdirs=file:///tmp/test/hadoop/TestConsumerPartitionStartTime/1/testclient,file:///tmp/test/hadoop/TestConsumerPartitionStartTime/2/testclient,file:///tmp/test/hadoop/TestConsumerPartitionStartTime/3/testclient +messaging.consumer.group.membership=1/2 diff --git a/messaging-client-databus/src/test/resources/messaging-consumer-hadoop-conf6.properties b/messaging-client-databus/src/test/resources/messaging-consumer-hadoop-conf6.properties new file mode 100644 index 0000000..7617332 --- /dev/null +++ b/messaging-client-databus/src/test/resources/messaging-consumer-hadoop-conf6.properties @@ -0,0 +1,10 @@ +consumer.classname=com.inmobi.messaging.consumer.hadoop.HadoopConsumer +topic.name=testclient +consumer.name=c1 +messaging.consumer.checkpoint.dir=/tmp/test/hadoop/TestConsumerPartitionStartTime/checkpoint +messaging.consumer.buffer.size=100 +messaging.consumer.inputformat.classname=org.apache.hadoop.mapred.SequenceFileInputFormat +messaging.consumer.topic.retention.inhours=4 +messaging.consumer.hadoop.conf=hadoop-conf.xml +hadoop.consumer.rootdirs=file:///tmp/test/hadoop/TestConsumerPartitionStartTime/1/testclient,file:///tmp/test/hadoop/TestConsumerPartitionStartTime/2/testclient,file:///tmp/test/hadoop/TestConsumerPartitionStartTime/3/testclient +messaging.consumer.group.membership=2/2 diff --git a/messaging-client-databus/src/test/resources/messaging-consumer-hadoop-conf7.properties b/messaging-client-databus/src/test/resources/messaging-consumer-hadoop-conf7.properties new file mode 100644 index 0000000..b56a3f7 --- /dev/null +++ b/messaging-client-databus/src/test/resources/messaging-consumer-hadoop-conf7.properties @@ -0,0 +1,10 @@ +consumer.classname=com.inmobi.messaging.consumer.hadoop.HadoopConsumer +topic.name=testclient +consumer.name=c1 +messaging.consumer.checkpoint.dir=/tmp/test/hadoop/8/checkpoint +messaging.consumer.buffer.size=100 +messaging.consumer.inputformat.classname=org.apache.hadoop.mapred.SequenceFileInputFormat +messaging.consumer.topic.retention.inhours=4 +messaging.consumer.hadoop.conf=hadoop-conf.xml +hadoop.consumer.rootdirs=file:///tmp/test/TestConsumerPartitionRetention/1/testStream,file:///tmp/test/TestConsumerPartitionRetention/2/testStream,file:///tmp/test/TestConsumerPartitionRetention/3/testStream +messaging.consumer.group.membership=1/2 diff --git a/messaging-client-databus/src/test/resources/messaging-consumer-hadoop-conf8.properties b/messaging-client-databus/src/test/resources/messaging-consumer-hadoop-conf8.properties new file mode 100644 index 0000000..963718b --- /dev/null +++ b/messaging-client-databus/src/test/resources/messaging-consumer-hadoop-conf8.properties @@ -0,0 +1,10 @@ +consumer.classname=com.inmobi.messaging.consumer.hadoop.HadoopConsumer +topic.name=testclient +consumer.name=c1 +messaging.consumer.checkpoint.dir=/tmp/test/hadoop/8/checkpoint +messaging.consumer.buffer.size=100 +messaging.consumer.inputformat.classname=org.apache.hadoop.mapred.SequenceFileInputFormat +messaging.consumer.topic.retention.inhours=4 +messaging.consumer.hadoop.conf=hadoop-conf.xml +hadoop.consumer.rootdirs=file:///tmp/test/TestConsumerPartitionRetention/1/testStream,file:///tmp/test/TestConsumerPartitionRetention/2/testStream,file:///tmp/test/TestConsumerPartitionRetention/3/testStream +messaging.consumer.group.membership=2/2