zookeeper-dev mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From afine <...@git.apache.org>
Subject [GitHub] zookeeper pull request #420: ZOOKEEPER-2924. Refactor tests of LoadFromLogTe...
Date Tue, 28 Nov 2017 23:14:42 GMT
Github user afine commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/420#discussion_r153652218
  
    --- Diff: src/java/test/org/apache/zookeeper/test/LoadFromLogTest.java ---
    @@ -18,77 +18,58 @@
     
     package org.apache.zookeeper.test;
     
    -import java.io.ByteArrayOutputStream;
     import java.io.File;
    -import java.io.FileInputStream;
     import java.io.IOException;
    -import java.nio.ByteBuffer;
    -import java.util.ArrayList;
    -import java.util.List;
    -
    -import org.apache.zookeeper.common.Time;
    -import org.apache.jute.BinaryInputArchive;
    -import org.apache.jute.BinaryOutputArchive;
    -import org.apache.jute.Record;
    +
     import org.apache.zookeeper.CreateMode;
     import org.apache.zookeeper.KeeperException.NoNodeException;
    -import org.apache.zookeeper.PortAssignment;
    -import org.apache.zookeeper.ZKTestCase;
     import org.apache.zookeeper.ZooDefs.Ids;
    -import org.apache.zookeeper.ZooDefs.OpCode;
     import org.apache.zookeeper.ZooKeeper;
     import org.apache.zookeeper.data.Stat;
    -import org.apache.zookeeper.server.DataNode;
    -import org.apache.zookeeper.server.DataTree;
     import org.apache.zookeeper.server.ServerCnxnFactory;
     import org.apache.zookeeper.server.SyncRequestProcessor;
     import org.apache.zookeeper.server.ZooKeeperServer;
    -import org.apache.zookeeper.server.persistence.FileHeader;
     import org.apache.zookeeper.server.persistence.FileTxnLog;
     import org.apache.zookeeper.server.persistence.FileTxnSnapLog;
     import org.apache.zookeeper.server.persistence.Util;
     import org.apache.zookeeper.server.persistence.FileTxnLog.FileTxnIterator;
     import org.apache.zookeeper.server.persistence.TxnLog.TxnIterator;
    -import org.apache.zookeeper.txn.CreateTxn;
    -import org.apache.zookeeper.txn.DeleteTxn;
    -import org.apache.zookeeper.txn.MultiTxn;
    -import org.apache.zookeeper.txn.Txn;
     import org.apache.zookeeper.txn.TxnHeader;
    +import org.eclipse.jetty.util.SocketAddressResolver;
    +import org.junit.After;
     import org.junit.Assert;
    +import org.junit.Before;
     import org.junit.Test;
     import org.slf4j.Logger;
     import org.slf4j.LoggerFactory;
     
    -public class LoadFromLogTest extends ZKTestCase {
    -    private static final String HOST = "127.0.0.1:";
    -    private static final int CONNECTION_TIMEOUT = 3000;
    +public class LoadFromLogTest extends ClientBase {
         private static final int NUM_MESSAGES = 300;
         protected static final Logger LOG = LoggerFactory.getLogger(LoadFromLogTest.class);
     
         // setting up the quorum has a transaction overhead for creating and closing the
session
         private static final int TRANSACTION_OVERHEAD = 2;
         private static final int TOTAL_TRANSACTIONS = NUM_MESSAGES + TRANSACTION_OVERHEAD;
     
    +    @Before
    +    public void setUp() throws Exception {
    +        SyncRequestProcessor.setSnapCount(100);
    --- End diff --
    
    I think @phunt is mostly correct. There are a couple issues with `snapCount`. 
    
    First, if want to change it while the syncProcessor thread has started, shouldn't it be
volatile? 
    
    Second, I disagree that it has "no effect" on a server that has already been started but
the effect is not the desired behavior. Taking the issue of the `snapCount` field not being
volatile out of the picture, the code in `SyncProcessor` looks like:
    ```java
        public void run() {
            try {
                int randRoll = r.nextInt(snapCount/2);
                while (true) {
                    Request si = null;
                    //get a request
                    if (si != null) {
                        // track the number of records written to the log
                        if (zks.getZKDatabase().append(si)) {
                            logCount++;
                            if (logCount > (snapCount / 2 + randRoll)) {
                                randRoll = r.nextInt(snapCount/2);
    ```
    
    So my reading is that if `snapCount` is changed from `previousSnapCount` while the `SyncProcessor`
is already running to `desiredSnapCount` it will take at least `desiredSnapCount` and at worst
`desiredSnapCount + previousSnapCount/2` transactions for the "effective" `snapCount` to become
`desiredSnapCount` (again ignoring the volatility issue).
    
    I agree with @phunt's solution.



---

Mime
View raw message