Skip to content

Commit

Permalink
Add ManagedLedgerTest
Browse files Browse the repository at this point in the history
  • Loading branch information
dao-jun committed Mar 9, 2024
1 parent e26d172 commit 1f0b134
Showing 1 changed file with 55 additions and 0 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -44,6 +44,7 @@
import com.google.common.collect.Sets;
import io.netty.buffer.ByteBuf;
import io.netty.buffer.ByteBufAllocator;
import io.netty.buffer.Unpooled;
import io.netty.util.concurrent.DefaultThreadFactory;
import java.lang.reflect.Field;
import java.nio.ReadOnlyBufferException;
Expand Down Expand Up @@ -4232,4 +4233,58 @@ public void testNoCleanupOffloadLedgerWhenMetadataExceptionHappens() throws Exce
verify(ledgerOffloader, times(0))
.deleteOffloaded(eq(ledgerInfo.getLedgerId()), any(), anyMap());
}

@Test
public void testLedgerPublishTime() throws Exception {
ManagedLedgerConfig config = new ManagedLedgerConfig();
config.setMaxEntriesPerLedger(2);
config.setMinimumRolloverTime(0, TimeUnit.SECONDS);
config.setRetentionTime(1, TimeUnit.DAYS);
config.setRetentionSizeInMB(1);
LinkedList<Long> publishTimestamps = new LinkedList<>();
@Cleanup
ManagedLedgerImpl ledger = (ManagedLedgerImpl) factory.open("testLedgerPublishTime", config);
CountDownLatch latch = new CountDownLatch(5);
for (int i = 0; i < 5; i++) {
Object ctx = new Entry.PublishTimestampProvider() {
@Override
public Long getPublishTimestamp() {
long timestamp = System.currentTimeMillis();
publishTimestamps.add(timestamp);
return timestamp;
}
};
ledger.asyncAddEntry(Unpooled.wrappedBuffer(("test-" + i).getBytes()), new AddEntryCallback() {

@Override
public void addComplete(Position position, ByteBuf entryData, Object ctx) {
latch.countDown();
}

@Override
public void addFailed(ManagedLedgerException exception, Object ctx) {
latch.countDown();
}
}, ctx);

Thread.sleep(10);
}

latch.await();
NavigableMap<Long, LedgerInfo> ledgers = ledger.getLedgersInfo();
assertEquals(ledger.getNumberOfEntries(), 5);
assertEquals(ledgers.size(), 3);
assertEquals(publishTimestamps.size(), 5);

LedgerInfo firstLedger = ledgers.firstEntry().getValue();
LedgerInfo secondLedger = ledgers.higherEntry(firstLedger.getLedgerId()).getValue();
LedgerInfo thirdLedger = ledgers.higherEntry(secondLedger.getLedgerId()).getValue();

assertTrue(firstLedger.getBeginPublishTimestamp() == publishTimestamps.get(0)
&& firstLedger.getEndPublishTimestamp() == publishTimestamps.get(1));
assertTrue(secondLedger.getBeginPublishTimestamp() == publishTimestamps.get(2)
&& secondLedger.getEndPublishTimestamp() == publishTimestamps.get(3));
assertTrue(thirdLedger.getBeginPublishTimestamp() == 0
&& thirdLedger.getEndPublishTimestamp() == 0);
}
}

0 comments on commit 1f0b134

Please sign in to comment.