accumulo-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From ctubb...@apache.org
Subject [08/24] accumulo git commit: Merge branch '1.7' into 1.8
Date Tue, 25 Jul 2017 23:02:56 GMT
http://git-wip-us.apache.org/repos/asf/accumulo/blob/018c7fe5/test/src/main/java/org/apache/accumulo/test/functional/YieldingIterator.java
----------------------------------------------------------------------
diff --cc test/src/main/java/org/apache/accumulo/test/functional/YieldingIterator.java
index 4ac325d,0000000..4ea771a
mode 100644,000000..100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/YieldingIterator.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/YieldingIterator.java
@@@ -1,129 -1,0 +1,128 @@@
 +/*
 + * Licensed to the Apache Software Foundation (ASF) under one or more
 + * contributor license agreements.  See the NOTICE file distributed with
 + * this work for additional information regarding copyright ownership.
 + * The ASF licenses this file to You under the Apache License, Version 2.0
 + * (the "License"); you may not use this file except in compliance with
 + * the License.  You may obtain a copy of the License at
 + *
 + *     http://www.apache.org/licenses/LICENSE-2.0
 + *
 + * Unless required by applicable law or agreed to in writing, software
 + * distributed under the License is distributed on an "AS IS" BASIS,
 + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 + * See the License for the specific language governing permissions and
 + * limitations under the License.
 + */
 +package org.apache.accumulo.test.functional;
 +
++import java.io.IOException;
++import java.util.Collection;
 +import java.util.concurrent.atomic.AtomicBoolean;
++import java.util.concurrent.atomic.AtomicInteger;
 +
- import com.google.common.base.Optional;
 +import org.apache.accumulo.core.data.ByteSequence;
 +import org.apache.accumulo.core.data.Key;
 +import org.apache.accumulo.core.data.PartialKey;
 +import org.apache.accumulo.core.data.Range;
 +import org.apache.accumulo.core.data.Value;
 +import org.apache.accumulo.core.iterators.IteratorEnvironment;
 +import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
 +import org.apache.accumulo.core.iterators.WrappingIterator;
- 
- import java.io.IOException;
- import java.util.Collection;
- import java.util.concurrent.atomic.AtomicInteger;
- 
 +import org.apache.accumulo.core.iterators.YieldCallback;
 +import org.apache.accumulo.core.iterators.YieldingKeyValueIterator;
 +import org.slf4j.Logger;
 +import org.slf4j.LoggerFactory;
 +
++import com.google.common.base.Optional;
++
 +/**
 + * This iterator which implements yielding will yield after every other next and every other seek call.
 + */
 +public class YieldingIterator extends WrappingIterator implements YieldingKeyValueIterator<Key,Value> {
 +  private static final Logger log = LoggerFactory.getLogger(YieldingIterator.class);
 +  private static final AtomicInteger yieldNexts = new AtomicInteger(0);
 +  private static final AtomicInteger yieldSeeks = new AtomicInteger(0);
 +  private static final AtomicInteger rebuilds = new AtomicInteger(0);
 +
 +  private static final AtomicBoolean yieldNextKey = new AtomicBoolean(false);
 +  private static final AtomicBoolean yieldSeekKey = new AtomicBoolean(false);
 +
 +  private Optional<YieldCallback<Key>> yield = Optional.absent();
 +
 +  @Override
 +  public SortedKeyValueIterator<Key,Value> deepCopy(IteratorEnvironment env) {
 +    YieldingIterator it = new YieldingIterator();
 +    it.setSource(getSource().deepCopy(env));
 +    return it;
 +  }
 +
 +  @Override
 +  public boolean hasTop() {
 +    return (!(yield.isPresent() && yield.get().hasYielded()) && super.hasTop());
 +  }
 +
 +  @Override
 +  public void next() throws IOException {
 +    log.info("start YieldingIterator.next: " + getTopValue());
 +    boolean yielded = false;
 +
 +    // yield on every other next call.
 +    yieldNextKey.set(!yieldNextKey.get());
 +    if (yield.isPresent() && yieldNextKey.get()) {
 +      yielded = true;
 +      yieldNexts.incrementAndGet();
 +      // since we are not actually skipping keys underneath, simply use the key following the top key as the yield key
 +      yield.get().yield(getTopKey().followingKey(PartialKey.ROW_COLFAM_COLQUAL_COLVIS_TIME));
 +      log.info("end YieldingIterator.next: yielded at " + getTopKey());
 +    }
 +
 +    // if not yielding, then simply pass on the next call
 +    if (!yielded) {
 +      super.next();
 +      log.info("end YieldingIterator.next: " + (hasTop() ? getTopKey() + " " + getTopValue() : "no top"));
 +    }
 +  }
 +
 +  /**
 +   * The top value will encode the current state of the yields, seeks, and rebuilds for use by the YieldScannersIT tests.
 +   *
 +   * @return a top value of the form {yieldNexts},{yieldSeeks},{rebuilds}
 +   */
 +  @Override
 +  public Value getTopValue() {
 +    String value = Integer.toString(yieldNexts.get()) + ',' + Integer.toString(yieldSeeks.get()) + ',' + Integer.toString(rebuilds.get());
 +    return new Value(value);
 +  }
 +
 +  @Override
 +  public void seek(Range range, Collection<ByteSequence> columnFamilies, boolean inclusive) throws IOException {
 +    log.info("start YieldingIterator.seek: " + getTopValue() + " with range " + range);
 +    boolean yielded = false;
 +
 +    if (!range.isStartKeyInclusive()) {
 +      rebuilds.incrementAndGet();
 +
 +      // yield on every other seek call.
 +      yieldSeekKey.set(!yieldSeekKey.get());
 +      if (yield.isPresent() && yieldSeekKey.get()) {
 +        yielded = true;
 +        yieldSeeks.incrementAndGet();
 +        // since we are not actually skipping keys underneath, simply use the key following the range start key
 +        yield.get().yield(range.getStartKey().followingKey(PartialKey.ROW_COLFAM_COLQUAL_COLVIS_TIME));
 +        log.info("end YieldingIterator.next: yielded at " + range.getStartKey());
 +      }
 +    }
 +
 +    // if not yielding, then simply pass on the call to the source
 +    if (!yielded) {
 +      super.seek(range, columnFamilies, inclusive);
 +      log.info("end YieldingIterator.seek: " + (hasTop() ? getTopKey() + " " + getTopValue() : "no top"));
 +    }
 +  }
 +
 +  @Override
 +  public void enableYielding(YieldCallback<Key> yield) {
 +    this.yield = Optional.of(yield);
 +  }
 +}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/018c7fe5/test/src/main/java/org/apache/accumulo/test/functional/ZookeeperRestartIT.java
----------------------------------------------------------------------
diff --cc test/src/main/java/org/apache/accumulo/test/functional/ZookeeperRestartIT.java
index 013edb0,0000000..e376555
mode 100644,000000..100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/ZookeeperRestartIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/ZookeeperRestartIT.java
@@@ -1,89 -1,0 +1,88 @@@
 +/*
 + * Licensed to the Apache Software Foundation (ASF) under one or more
 + * contributor license agreements.  See the NOTICE file distributed with
 + * this work for additional information regarding copyright ownership.
 + * The ASF licenses this file to You under the Apache License, Version 2.0
 + * (the "License"); you may not use this file except in compliance with
 + * the License.  You may obtain a copy of the License at
 + *
 + *     http://www.apache.org/licenses/LICENSE-2.0
 + *
 + * Unless required by applicable law or agreed to in writing, software
 + * distributed under the License is distributed on an "AS IS" BASIS,
 + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 + * See the License for the specific language governing permissions and
 + * limitations under the License.
 + */
 +package org.apache.accumulo.test.functional;
 +
++import static com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly;
 +import static org.junit.Assert.assertEquals;
 +import static org.junit.Assert.assertFalse;
 +import static org.junit.Assert.assertTrue;
 +
 +import java.util.HashMap;
 +import java.util.Iterator;
 +import java.util.Map;
 +import java.util.Map.Entry;
 +import java.util.concurrent.TimeUnit;
 +
 +import org.apache.accumulo.core.client.BatchWriter;
 +import org.apache.accumulo.core.client.Connector;
 +import org.apache.accumulo.core.client.Scanner;
 +import org.apache.accumulo.core.conf.Property;
 +import org.apache.accumulo.core.data.Key;
 +import org.apache.accumulo.core.data.Mutation;
 +import org.apache.accumulo.core.data.Value;
 +import org.apache.accumulo.core.security.Authorizations;
 +import org.apache.accumulo.minicluster.ServerType;
 +import org.apache.accumulo.minicluster.impl.MiniAccumuloConfigImpl;
 +import org.apache.accumulo.minicluster.impl.ProcessReference;
 +import org.apache.hadoop.conf.Configuration;
 +import org.junit.Test;
 +
- import static com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly;
- 
 +public class ZookeeperRestartIT extends ConfigurableMacBase {
 +
 +  @Override
 +  public void configure(MiniAccumuloConfigImpl cfg, Configuration hadoopCoreSite) {
 +    Map<String,String> siteConfig = new HashMap<>();
 +    siteConfig.put(Property.INSTANCE_ZK_TIMEOUT.getKey(), "15s");
 +    cfg.setSiteConfig(siteConfig);
 +  }
 +
 +  @Override
 +  protected int defaultTimeoutSeconds() {
 +    return 2 * 60;
 +  }
 +
 +  @Test
 +  public void test() throws Exception {
 +    Connector c = getConnector();
 +    c.tableOperations().create("test_ingest");
 +    BatchWriter bw = c.createBatchWriter("test_ingest", null);
 +    Mutation m = new Mutation("row");
 +    m.put("cf", "cq", "value");
 +    bw.addMutation(m);
 +    bw.close();
 +
 +    // kill zookeeper
 +    for (ProcessReference proc : cluster.getProcesses().get(ServerType.ZOOKEEPER))
 +      cluster.killProcess(ServerType.ZOOKEEPER, proc);
 +
 +    // give the servers time to react
 +    sleepUninterruptibly(1, TimeUnit.SECONDS);
 +
 +    // start zookeeper back up
 +    cluster.start();
 +
 +    // use the tservers
 +    Scanner s = c.createScanner("test_ingest", Authorizations.EMPTY);
 +    Iterator<Entry<Key,Value>> i = s.iterator();
 +    assertTrue(i.hasNext());
 +    assertEquals("row", i.next().getKey().getRow().toString());
 +    assertFalse(i.hasNext());
 +    // use the master
 +    c.tableOperations().delete("test_ingest");
 +  }
 +
 +}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/018c7fe5/test/src/main/java/org/apache/accumulo/test/gc/replication/CloseWriteAheadLogReferencesIT.java
----------------------------------------------------------------------
diff --cc test/src/main/java/org/apache/accumulo/test/gc/replication/CloseWriteAheadLogReferencesIT.java
index 5af9ebe,0000000..352914d
mode 100644,000000..100644
--- a/test/src/main/java/org/apache/accumulo/test/gc/replication/CloseWriteAheadLogReferencesIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/gc/replication/CloseWriteAheadLogReferencesIT.java
@@@ -1,184 -1,0 +1,183 @@@
- package org.apache.accumulo.test.gc.replication;
- 
 +/*
 + * Licensed to the Apache Software Foundation (ASF) under one or more
 + * contributor license agreements.  See the NOTICE file distributed with
 + * this work for additional information regarding copyright ownership.
 + * The ASF licenses this file to You under the Apache License, Version 2.0
 + * (the "License"); you may not use this file except in compliance with
 + * the License.  You may obtain a copy of the License at
 + *
 + *     http://www.apache.org/licenses/LICENSE-2.0
 + *
 + * Unless required by applicable law or agreed to in writing, software
 + * distributed under the License is distributed on an "AS IS" BASIS,
 + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 + * See the License for the specific language governing permissions and
 + * limitations under the License.
 + */
++package org.apache.accumulo.test.gc.replication;
 +
 +import static org.easymock.EasyMock.createMock;
 +import static org.easymock.EasyMock.expect;
 +import static org.easymock.EasyMock.replay;
 +
 +import java.util.Collections;
 +import java.util.HashMap;
 +import java.util.Iterator;
 +import java.util.Map.Entry;
 +import java.util.Set;
 +
 +import org.apache.accumulo.core.client.BatchWriter;
 +import org.apache.accumulo.core.client.BatchWriterConfig;
 +import org.apache.accumulo.core.client.Connector;
 +import org.apache.accumulo.core.client.Instance;
 +import org.apache.accumulo.core.client.Scanner;
 +import org.apache.accumulo.core.conf.AccumuloConfiguration;
 +import org.apache.accumulo.core.conf.ConfigurationCopy;
 +import org.apache.accumulo.core.conf.Property;
 +import org.apache.accumulo.core.conf.SiteConfiguration;
 +import org.apache.accumulo.core.data.Key;
 +import org.apache.accumulo.core.data.Mutation;
 +import org.apache.accumulo.core.data.Value;
 +import org.apache.accumulo.core.metadata.MetadataTable;
 +import org.apache.accumulo.core.metadata.schema.MetadataSchema.ReplicationSection;
 +import org.apache.accumulo.core.protobuf.ProtobufUtil;
 +import org.apache.accumulo.core.replication.ReplicationSchema.StatusSection;
 +import org.apache.accumulo.core.replication.ReplicationTable;
 +import org.apache.accumulo.core.security.Authorizations;
 +import org.apache.accumulo.core.security.TablePermission;
 +import org.apache.accumulo.gc.replication.CloseWriteAheadLogReferences;
 +import org.apache.accumulo.server.AccumuloServerContext;
 +import org.apache.accumulo.server.conf.ServerConfigurationFactory;
 +import org.apache.accumulo.server.replication.StatusUtil;
 +import org.apache.accumulo.server.replication.proto.Replication.Status;
 +import org.apache.accumulo.test.functional.ConfigurableMacBase;
 +import org.apache.hadoop.io.Text;
 +import org.easymock.EasyMock;
 +import org.easymock.IAnswer;
 +import org.junit.Assert;
 +import org.junit.Before;
 +import org.junit.Test;
 +
 +import com.google.common.collect.Iterables;
 +
 +public class CloseWriteAheadLogReferencesIT extends ConfigurableMacBase {
 +
 +  private WrappedCloseWriteAheadLogReferences refs;
 +  private Connector conn;
 +
 +  private static class WrappedCloseWriteAheadLogReferences extends CloseWriteAheadLogReferences {
 +    public WrappedCloseWriteAheadLogReferences(AccumuloServerContext context) {
 +      super(context);
 +    }
 +
 +    @Override
 +    protected long updateReplicationEntries(Connector conn, Set<String> closedWals) {
 +      return super.updateReplicationEntries(conn, closedWals);
 +    }
 +  }
 +
 +  @Before
 +  public void setupInstance() throws Exception {
 +    conn = getConnector();
 +    conn.securityOperations().grantTablePermission(conn.whoami(), ReplicationTable.NAME, TablePermission.WRITE);
 +    conn.securityOperations().grantTablePermission(conn.whoami(), MetadataTable.NAME, TablePermission.WRITE);
 +    ReplicationTable.setOnline(conn);
 +  }
 +
 +  @Before
 +  public void setupEasyMockStuff() {
 +    Instance mockInst = createMock(Instance.class);
 +    SiteConfiguration siteConfig = EasyMock.createMock(SiteConfiguration.class);
 +    expect(mockInst.getInstanceID()).andReturn(testName.getMethodName()).anyTimes();
 +    expect(mockInst.getZooKeepers()).andReturn("localhost").anyTimes();
 +    expect(mockInst.getZooKeepersSessionTimeOut()).andReturn(30000).anyTimes();
 +    final AccumuloConfiguration systemConf = new ConfigurationCopy(new HashMap<String,String>());
 +    ServerConfigurationFactory factory = createMock(ServerConfigurationFactory.class);
 +    expect(factory.getConfiguration()).andReturn(systemConf).anyTimes();
 +    expect(factory.getInstance()).andReturn(mockInst).anyTimes();
 +    expect(factory.getSiteConfiguration()).andReturn(siteConfig).anyTimes();
 +
 +    // Just make the SiteConfiguration delegate to our AccumuloConfiguration
 +    // Presently, we only need get(Property) and iterator().
 +    EasyMock.expect(siteConfig.get(EasyMock.anyObject(Property.class))).andAnswer(new IAnswer<String>() {
 +      @Override
 +      public String answer() {
 +        Object[] args = EasyMock.getCurrentArguments();
 +        return systemConf.get((Property) args[0]);
 +      }
 +    }).anyTimes();
 +    EasyMock.expect(siteConfig.getBoolean(EasyMock.anyObject(Property.class))).andAnswer(new IAnswer<Boolean>() {
 +      @Override
 +      public Boolean answer() {
 +        Object[] args = EasyMock.getCurrentArguments();
 +        return systemConf.getBoolean((Property) args[0]);
 +      }
 +    }).anyTimes();
 +
 +    EasyMock.expect(siteConfig.iterator()).andAnswer(new IAnswer<Iterator<Entry<String,String>>>() {
 +      @Override
 +      public Iterator<Entry<String,String>> answer() {
 +        return systemConf.iterator();
 +      }
 +    }).anyTimes();
 +
 +    replay(mockInst, factory, siteConfig);
 +    refs = new WrappedCloseWriteAheadLogReferences(new AccumuloServerContext(factory));
 +  }
 +
 +  @Test
 +  public void unclosedWalsLeaveStatusOpen() throws Exception {
 +    Set<String> wals = Collections.emptySet();
 +    BatchWriter bw = conn.createBatchWriter(MetadataTable.NAME, new BatchWriterConfig());
 +    Mutation m = new Mutation(ReplicationSection.getRowPrefix() + "file:/accumulo/wal/tserver+port/12345");
 +    m.put(ReplicationSection.COLF, new Text("1"), StatusUtil.fileCreatedValue(System.currentTimeMillis()));
 +    bw.addMutation(m);
 +    bw.close();
 +
 +    refs.updateReplicationEntries(conn, wals);
 +
 +    Scanner s = conn.createScanner(MetadataTable.NAME, Authorizations.EMPTY);
 +    s.fetchColumnFamily(ReplicationSection.COLF);
 +    Entry<Key,Value> entry = Iterables.getOnlyElement(s);
 +    Status status = Status.parseFrom(entry.getValue().get());
 +    Assert.assertFalse(status.getClosed());
 +  }
 +
 +  @Test
 +  public void closedWalsUpdateStatus() throws Exception {
 +    String file = "file:/accumulo/wal/tserver+port/12345";
 +    Set<String> wals = Collections.singleton(file);
 +    BatchWriter bw = conn.createBatchWriter(MetadataTable.NAME, new BatchWriterConfig());
 +    Mutation m = new Mutation(ReplicationSection.getRowPrefix() + file);
 +    m.put(ReplicationSection.COLF, new Text("1"), StatusUtil.fileCreatedValue(System.currentTimeMillis()));
 +    bw.addMutation(m);
 +    bw.close();
 +
 +    refs.updateReplicationEntries(conn, wals);
 +
 +    Scanner s = conn.createScanner(MetadataTable.NAME, Authorizations.EMPTY);
 +    s.fetchColumnFamily(ReplicationSection.COLF);
 +    Entry<Key,Value> entry = Iterables.getOnlyElement(s);
 +    Status status = Status.parseFrom(entry.getValue().get());
 +    Assert.assertTrue(status.getClosed());
 +  }
 +
 +  @Test
 +  public void partiallyReplicatedReferencedWalsAreNotClosed() throws Exception {
 +    String file = "file:/accumulo/wal/tserver+port/12345";
 +    Set<String> wals = Collections.singleton(file);
 +    BatchWriter bw = ReplicationTable.getBatchWriter(conn);
 +    Mutation m = new Mutation(file);
 +    StatusSection.add(m, "1", ProtobufUtil.toValue(StatusUtil.ingestedUntil(1000)));
 +    bw.addMutation(m);
 +    bw.close();
 +
 +    refs.updateReplicationEntries(conn, wals);
 +
 +    Scanner s = ReplicationTable.getScanner(conn);
 +    Entry<Key,Value> entry = Iterables.getOnlyElement(s);
 +    Status status = Status.parseFrom(entry.getValue().get());
 +    Assert.assertFalse(status.getClosed());
 +  }
 +
 +}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/018c7fe5/test/src/main/java/org/apache/accumulo/test/proxy/ProxyDurabilityIT.java
----------------------------------------------------------------------
diff --cc test/src/main/java/org/apache/accumulo/test/proxy/ProxyDurabilityIT.java
index d9a1027,0000000..c1115ab
mode 100644,000000..100644
--- a/test/src/main/java/org/apache/accumulo/test/proxy/ProxyDurabilityIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/proxy/ProxyDurabilityIT.java
@@@ -1,146 -1,0 +1,146 @@@
 +/*
 + * Licensed to the Apache Software Foundation (ASF) under one or more
 + * contributor license agreements.  See the NOTICE file distributed with
 + * this work for additional information regarding copyright ownership.
 + * The ASF licenses this file to You under the Apache License, Version 2.0
 + * (the "License"); you may not use this file except in compliance with
 + * the License.  You may obtain a copy of the License at
 + *
 + *     http://www.apache.org/licenses/LICENSE-2.0
 + *
 + * Unless required by applicable law or agreed to in writing, software
 + * distributed under the License is distributed on an "AS IS" BASIS,
 + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 + * See the License for the specific language governing permissions and
 + * limitations under the License.
 + */
 +package org.apache.accumulo.test.proxy;
 +
++import static com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly;
 +import static org.junit.Assert.assertEquals;
 +import static org.junit.Assert.assertTrue;
 +
 +import java.io.File;
 +import java.nio.ByteBuffer;
 +import java.nio.file.Files;
 +import java.util.Collections;
 +import java.util.List;
 +import java.util.Map;
 +import java.util.Properties;
 +import java.util.TreeMap;
 +import java.util.concurrent.TimeUnit;
 +
 +import org.apache.accumulo.core.client.Connector;
 +import org.apache.accumulo.core.client.security.tokens.PasswordToken;
 +import org.apache.accumulo.core.conf.Property;
 +import org.apache.accumulo.core.security.Authorizations;
 +import org.apache.accumulo.minicluster.ServerType;
 +import org.apache.accumulo.minicluster.impl.MiniAccumuloConfigImpl;
 +import org.apache.accumulo.minicluster.impl.ProcessReference;
 +import org.apache.accumulo.proxy.Proxy;
 +import org.apache.accumulo.proxy.thrift.AccumuloProxy.Client;
 +import org.apache.accumulo.proxy.thrift.Column;
 +import org.apache.accumulo.proxy.thrift.ColumnUpdate;
 +import org.apache.accumulo.proxy.thrift.Condition;
 +import org.apache.accumulo.proxy.thrift.ConditionalStatus;
 +import org.apache.accumulo.proxy.thrift.ConditionalUpdates;
 +import org.apache.accumulo.proxy.thrift.ConditionalWriterOptions;
 +import org.apache.accumulo.proxy.thrift.Durability;
 +import org.apache.accumulo.proxy.thrift.TimeType;
 +import org.apache.accumulo.proxy.thrift.WriterOptions;
 +import org.apache.accumulo.server.util.PortUtils;
 +import org.apache.accumulo.test.functional.ConfigurableMacBase;
 +import org.apache.hadoop.conf.Configuration;
 +import org.apache.hadoop.fs.RawLocalFileSystem;
 +import org.apache.thrift.protocol.TJSONProtocol;
 +import org.apache.thrift.server.TServer;
 +import org.junit.Test;
 +
 +import com.google.common.collect.Iterators;
 +import com.google.common.net.HostAndPort;
- import static com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly;
 +
 +public class ProxyDurabilityIT extends ConfigurableMacBase {
 +
 +  @Override
 +  protected int defaultTimeoutSeconds() {
 +    return 120;
 +  }
 +
 +  @Override
 +  public void configure(MiniAccumuloConfigImpl cfg, Configuration hadoopCoreSite) {
 +    hadoopCoreSite.set("fs.file.impl", RawLocalFileSystem.class.getName());
 +    cfg.setProperty(Property.INSTANCE_ZK_TIMEOUT, "15s");
 +    cfg.setNumTservers(1);
 +  }
 +
 +  private static ByteBuffer bytes(String value) {
 +    return ByteBuffer.wrap(value.getBytes());
 +  }
 +
 +  @Test
 +  public void testDurability() throws Exception {
 +    Connector c = getConnector();
 +    Properties props = new Properties();
 +    // Avoid issues with locally installed client configuration files with custom properties
 +    File emptyFile = Files.createTempFile(null, null).toFile();
 +    emptyFile.deleteOnExit();
 +    props.put("instance", c.getInstance().getInstanceName());
 +    props.put("zookeepers", c.getInstance().getZooKeepers());
 +    props.put("tokenClass", PasswordToken.class.getName());
 +    props.put("clientConfigurationFile", emptyFile.toString());
 +
 +    TJSONProtocol.Factory protocol = new TJSONProtocol.Factory();
 +
 +    int proxyPort = PortUtils.getRandomFreePort();
 +    final TServer proxyServer = Proxy.createProxyServer(HostAndPort.fromParts("localhost", proxyPort), protocol, props).server;
 +    while (!proxyServer.isServing())
 +      sleepUninterruptibly(100, TimeUnit.MILLISECONDS);
 +    Client client = new TestProxyClient("localhost", proxyPort, protocol).proxy();
 +    Map<String,String> properties = new TreeMap<>();
 +    properties.put("password", ROOT_PASSWORD);
 +    ByteBuffer login = client.login("root", properties);
 +
 +    String tableName = getUniqueNames(1)[0];
 +    client.createTable(login, tableName, true, TimeType.MILLIS);
 +    assertTrue(c.tableOperations().exists(tableName));
 +
 +    WriterOptions options = new WriterOptions();
 +    options.setDurability(Durability.NONE);
 +    String writer = client.createWriter(login, tableName, options);
 +    Map<ByteBuffer,List<ColumnUpdate>> cells = new TreeMap<>();
 +    ColumnUpdate column = new ColumnUpdate(bytes("cf"), bytes("cq"));
 +    column.setValue("value".getBytes());
 +    cells.put(bytes("row"), Collections.singletonList(column));
 +    client.update(writer, cells);
 +    client.closeWriter(writer);
 +    assertEquals(1, count(tableName));
 +    restartTServer();
 +    assertEquals(0, count(tableName));
 +
 +    ConditionalWriterOptions cfg = new ConditionalWriterOptions();
 +    cfg.setDurability(Durability.SYNC);
 +    String cwriter = client.createConditionalWriter(login, tableName, cfg);
 +    ConditionalUpdates updates = new ConditionalUpdates();
 +    updates.addToConditions(new Condition(new Column(bytes("cf"), bytes("cq"), bytes(""))));
 +    updates.addToUpdates(column);
 +    Map<ByteBuffer,ConditionalStatus> status = client.updateRowsConditionally(cwriter, Collections.singletonMap(bytes("row"), updates));
 +    assertEquals(ConditionalStatus.ACCEPTED, status.get(bytes("row")));
 +    assertEquals(1, count(tableName));
 +    restartTServer();
 +    assertEquals(1, count(tableName));
 +
 +    proxyServer.stop();
 +  }
 +
 +  private void restartTServer() throws Exception {
 +    for (ProcessReference proc : cluster.getProcesses().get(ServerType.TABLET_SERVER)) {
 +      cluster.killProcess(ServerType.TABLET_SERVER, proc);
 +    }
 +    cluster.start();
 +  }
 +
 +  private int count(String tableName) throws Exception {
 +    return Iterators.size((getConnector().createScanner(tableName, Authorizations.EMPTY)).iterator());
 +  }
 +
 +}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/018c7fe5/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/OfflineTable.java
----------------------------------------------------------------------
diff --cc test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/OfflineTable.java
index b5af0b7,ba6389f..d06ad9a
--- a/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/OfflineTable.java
+++ b/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/OfflineTable.java
@@@ -16,6 -16,6 +16,8 @@@
   */
  package org.apache.accumulo.test.randomwalk.concurrent;
  
++import static com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly;
++
  import java.util.List;
  import java.util.Properties;
  import java.util.Random;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/018c7fe5/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/Replication.java
----------------------------------------------------------------------
diff --cc test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/Replication.java
index 9f3e0aa,c1b2502..25b0ccc
--- a/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/Replication.java
+++ b/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/Replication.java
@@@ -16,6 -16,6 +16,7 @@@
   */
  package org.apache.accumulo.test.randomwalk.concurrent;
  
++import static com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly;
  import static org.apache.accumulo.core.conf.Property.MASTER_REPLICATION_SCAN_INTERVAL;
  import static org.apache.accumulo.core.conf.Property.REPLICATION_NAME;
  import static org.apache.accumulo.core.conf.Property.REPLICATION_PEERS;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/018c7fe5/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/Shutdown.java
----------------------------------------------------------------------
diff --cc test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/Shutdown.java
index eeaeea6,6cc8312..3887a2a
--- a/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/Shutdown.java
+++ b/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/Shutdown.java
@@@ -16,8 -16,7 +16,10 @@@
   */
  package org.apache.accumulo.test.randomwalk.concurrent;
  
++import static com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly;
++
  import java.util.Properties;
 +import java.util.concurrent.TimeUnit;
  
  import org.apache.accumulo.core.client.impl.MasterClient;
  import org.apache.accumulo.core.master.thrift.MasterClientService.Client;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/018c7fe5/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/StartAll.java
----------------------------------------------------------------------
diff --cc test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/StartAll.java
index cfc0053,8504fd1..64d9abd
--- a/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/StartAll.java
+++ b/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/StartAll.java
@@@ -16,8 -16,7 +16,10 @@@
   */
  package org.apache.accumulo.test.randomwalk.concurrent;
  
++import static com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly;
++
  import java.util.Properties;
 +import java.util.concurrent.TimeUnit;
  
  import org.apache.accumulo.core.client.impl.MasterClient;
  import org.apache.accumulo.core.master.thrift.MasterClientService.Client;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/018c7fe5/test/src/main/java/org/apache/accumulo/test/randomwalk/shard/BulkInsert.java
----------------------------------------------------------------------
diff --cc test/src/main/java/org/apache/accumulo/test/randomwalk/shard/BulkInsert.java
index 5f696d0,095b5f7..3ec2d6d
--- a/test/src/main/java/org/apache/accumulo/test/randomwalk/shard/BulkInsert.java
+++ b/test/src/main/java/org/apache/accumulo/test/randomwalk/shard/BulkInsert.java
@@@ -16,6 -16,6 +16,7 @@@
   */
  package org.apache.accumulo.test.randomwalk.shard;
  
++import static com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly;
  import static java.nio.charset.StandardCharsets.UTF_8;
  
  import java.io.BufferedOutputStream;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/018c7fe5/test/src/main/java/org/apache/accumulo/test/replication/FinishedWorkUpdaterIT.java
----------------------------------------------------------------------
diff --cc test/src/main/java/org/apache/accumulo/test/replication/FinishedWorkUpdaterIT.java
index 5519013,0000000..f635f69
mode 100644,000000..100644
--- a/test/src/main/java/org/apache/accumulo/test/replication/FinishedWorkUpdaterIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/replication/FinishedWorkUpdaterIT.java
@@@ -1,173 -1,0 +1,172 @@@
- package org.apache.accumulo.test.replication;
- 
 +/*
 + * Licensed to the Apache Software Foundation (ASF) under one or more
 + * contributor license agreements.  See the NOTICE file distributed with
 + * this work for additional information regarding copyright ownership.
 + * The ASF licenses this file to You under the Apache License, Version 2.0
 + * (the "License"); you may not use this file except in compliance with
 + * the License.  You may obtain a copy of the License at
 + *
 + *     http://www.apache.org/licenses/LICENSE-2.0
 + *
 + * Unless required by applicable law or agreed to in writing, software
 + * distributed under the License is distributed on an "AS IS" BASIS,
 + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 + * See the License for the specific language governing permissions and
 + * limitations under the License.
 + */
++package org.apache.accumulo.test.replication;
 +
 +import java.util.Map.Entry;
 +import java.util.UUID;
 +
 +import org.apache.accumulo.core.client.BatchWriter;
 +import org.apache.accumulo.core.client.Connector;
 +import org.apache.accumulo.core.client.Scanner;
 +import org.apache.accumulo.core.data.Key;
 +import org.apache.accumulo.core.data.Mutation;
 +import org.apache.accumulo.core.data.Range;
 +import org.apache.accumulo.core.data.Value;
 +import org.apache.accumulo.core.protobuf.ProtobufUtil;
 +import org.apache.accumulo.core.replication.ReplicationSchema.StatusSection;
 +import org.apache.accumulo.core.replication.ReplicationSchema.WorkSection;
 +import org.apache.accumulo.core.replication.ReplicationTable;
 +import org.apache.accumulo.core.replication.ReplicationTarget;
 +import org.apache.accumulo.core.security.TablePermission;
 +import org.apache.accumulo.master.replication.FinishedWorkUpdater;
 +import org.apache.accumulo.server.replication.proto.Replication.Status;
 +import org.apache.accumulo.test.functional.ConfigurableMacBase;
 +import org.junit.Assert;
 +import org.junit.Before;
 +import org.junit.Test;
 +
 +import com.google.common.collect.Iterables;
 +
 +public class FinishedWorkUpdaterIT extends ConfigurableMacBase {
 +
 +  private Connector conn;
 +  private FinishedWorkUpdater updater;
 +
 +  @Before
 +  public void configureUpdater() throws Exception {
 +    conn = getConnector();
 +    updater = new FinishedWorkUpdater(conn);
 +  }
 +
 +  @Test
 +  public void offlineReplicationTableFailsGracefully() {
 +    updater.run();
 +  }
 +
 +  @Test
 +  public void recordsWithProgressUpdateBothTables() throws Exception {
 +    conn.securityOperations().grantTablePermission(conn.whoami(), ReplicationTable.NAME, TablePermission.READ);
 +    conn.securityOperations().grantTablePermission(conn.whoami(), ReplicationTable.NAME, TablePermission.WRITE);
 +    ReplicationTable.setOnline(conn);
 +
 +    String file = "/accumulo/wals/tserver+port/" + UUID.randomUUID();
 +    Status stat = Status.newBuilder().setBegin(100).setEnd(200).setClosed(true).setInfiniteEnd(false).build();
 +    ReplicationTarget target = new ReplicationTarget("peer", "table1", "1");
 +
 +    // Create a single work record for a file to some peer
 +    BatchWriter bw = ReplicationTable.getBatchWriter(conn);
 +    Mutation m = new Mutation(file);
 +    WorkSection.add(m, target.toText(), ProtobufUtil.toValue(stat));
 +    bw.addMutation(m);
 +    bw.close();
 +
 +    updater.run();
 +
 +    Scanner s = ReplicationTable.getScanner(conn);
 +    s.setRange(Range.exact(file));
 +    StatusSection.limit(s);
 +    Entry<Key,Value> entry = Iterables.getOnlyElement(s);
 +
 +    Assert.assertEquals(entry.getKey().getColumnFamily(), StatusSection.NAME);
 +    Assert.assertEquals(entry.getKey().getColumnQualifier().toString(), target.getSourceTableId());
 +
 +    // We should only rely on the correct begin attribute being returned
 +    Status actual = Status.parseFrom(entry.getValue().get());
 +    Assert.assertEquals(stat.getBegin(), actual.getBegin());
 +  }
 +
 +  @Test
 +  public void chooseMinimumBeginOffset() throws Exception {
 +    conn.securityOperations().grantTablePermission(conn.whoami(), ReplicationTable.NAME, TablePermission.READ);
 +    conn.securityOperations().grantTablePermission(conn.whoami(), ReplicationTable.NAME, TablePermission.WRITE);
 +    ReplicationTable.setOnline(conn);
 +
 +    String file = "/accumulo/wals/tserver+port/" + UUID.randomUUID();
 +    // @formatter:off
 +    Status stat1 = Status.newBuilder().setBegin(100).setEnd(1000).setClosed(true).setInfiniteEnd(false).build(),
 +        stat2 = Status.newBuilder().setBegin(500).setEnd(1000).setClosed(true).setInfiniteEnd(false).build(),
 +        stat3 = Status.newBuilder().setBegin(1).setEnd(1000).setClosed(true).setInfiniteEnd(false).build();
 +    ReplicationTarget target1 = new ReplicationTarget("peer1", "table1", "1"),
 +        target2 = new ReplicationTarget("peer2", "table2", "1"),
 +        target3 = new ReplicationTarget("peer3", "table3", "1");
 +    // @formatter:on
 +
 +    // Create a single work record for a file to some peer
 +    BatchWriter bw = ReplicationTable.getBatchWriter(conn);
 +    Mutation m = new Mutation(file);
 +    WorkSection.add(m, target1.toText(), ProtobufUtil.toValue(stat1));
 +    WorkSection.add(m, target2.toText(), ProtobufUtil.toValue(stat2));
 +    WorkSection.add(m, target3.toText(), ProtobufUtil.toValue(stat3));
 +    bw.addMutation(m);
 +    bw.close();
 +
 +    updater.run();
 +
 +    Scanner s = ReplicationTable.getScanner(conn);
 +    s.setRange(Range.exact(file));
 +    StatusSection.limit(s);
 +    Entry<Key,Value> entry = Iterables.getOnlyElement(s);
 +
 +    Assert.assertEquals(entry.getKey().getColumnFamily(), StatusSection.NAME);
 +    Assert.assertEquals(entry.getKey().getColumnQualifier().toString(), target1.getSourceTableId());
 +
 +    // We should only rely on the correct begin attribute being returned
 +    Status actual = Status.parseFrom(entry.getValue().get());
 +    Assert.assertEquals(1, actual.getBegin());
 +  }
 +
 +  @Test
 +  public void chooseMinimumBeginOffsetInfiniteEnd() throws Exception {
 +    conn.securityOperations().grantTablePermission(conn.whoami(), ReplicationTable.NAME, TablePermission.READ);
 +    conn.securityOperations().grantTablePermission(conn.whoami(), ReplicationTable.NAME, TablePermission.WRITE);
 +    ReplicationTable.setOnline(conn);
 +
 +    String file = "/accumulo/wals/tserver+port/" + UUID.randomUUID();
 +    // @formatter:off
 +    Status stat1 = Status.newBuilder().setBegin(100).setEnd(1000).setClosed(true).setInfiniteEnd(true).build(),
 +        stat2 = Status.newBuilder().setBegin(1).setEnd(1000).setClosed(true).setInfiniteEnd(true).build(),
 +        stat3 = Status.newBuilder().setBegin(500).setEnd(1000).setClosed(true).setInfiniteEnd(true).build();
 +    ReplicationTarget target1 = new ReplicationTarget("peer1", "table1", "1"),
 +        target2 = new ReplicationTarget("peer2", "table2", "1"),
 +        target3 = new ReplicationTarget("peer3", "table3", "1");
 +    // @formatter:on
 +
 +    // Create a single work record for a file to some peer
 +    BatchWriter bw = ReplicationTable.getBatchWriter(conn);
 +    Mutation m = new Mutation(file);
 +    WorkSection.add(m, target1.toText(), ProtobufUtil.toValue(stat1));
 +    WorkSection.add(m, target2.toText(), ProtobufUtil.toValue(stat2));
 +    WorkSection.add(m, target3.toText(), ProtobufUtil.toValue(stat3));
 +    bw.addMutation(m);
 +    bw.close();
 +
 +    updater.run();
 +
 +    Scanner s = ReplicationTable.getScanner(conn);
 +    s.setRange(Range.exact(file));
 +    StatusSection.limit(s);
 +    Entry<Key,Value> entry = Iterables.getOnlyElement(s);
 +
 +    Assert.assertEquals(entry.getKey().getColumnFamily(), StatusSection.NAME);
 +    Assert.assertEquals(entry.getKey().getColumnQualifier().toString(), target1.getSourceTableId());
 +
 +    // We should only rely on the correct begin attribute being returned
 +    Status actual = Status.parseFrom(entry.getValue().get());
 +    Assert.assertEquals(1, actual.getBegin());
 +  }
 +
 +}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/018c7fe5/test/src/main/java/org/apache/accumulo/test/replication/MultiInstanceReplicationIT.java
----------------------------------------------------------------------
diff --cc test/src/main/java/org/apache/accumulo/test/replication/MultiInstanceReplicationIT.java
index 33e0a55,0000000..2852c70
mode 100644,000000..100644
--- a/test/src/main/java/org/apache/accumulo/test/replication/MultiInstanceReplicationIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/replication/MultiInstanceReplicationIT.java
@@@ -1,733 -1,0 +1,734 @@@
 +/*
 + * Licensed to the Apache Software Foundation (ASF) under one or more
 + * contributor license agreements.  See the NOTICE file distributed with
 + * this work for additional information regarding copyright ownership.
 + * The ASF licenses this file to You under the Apache License, Version 2.0
 + * (the "License"); you may not use this file except in compliance with
 + * the License.  You may obtain a copy of the License at
 + *
 + *     http://www.apache.org/licenses/LICENSE-2.0
 + *
 + * Unless required by applicable law or agreed to in writing, software
 + * distributed under the License is distributed on an "AS IS" BASIS,
 + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 + * See the License for the specific language governing permissions and
 + * limitations under the License.
 + */
 +package org.apache.accumulo.test.replication;
 +
++import static com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly;
++
 +import java.util.HashMap;
 +import java.util.Iterator;
 +import java.util.Map;
 +import java.util.Map.Entry;
 +import java.util.Set;
 +import java.util.concurrent.Callable;
 +import java.util.concurrent.ExecutorService;
 +import java.util.concurrent.Executors;
 +import java.util.concurrent.Future;
 +import java.util.concurrent.TimeUnit;
 +import java.util.concurrent.TimeoutException;
 +
 +import org.apache.accumulo.core.client.BatchWriter;
 +import org.apache.accumulo.core.client.BatchWriterConfig;
 +import org.apache.accumulo.core.client.Connector;
 +import org.apache.accumulo.core.client.Scanner;
 +import org.apache.accumulo.core.client.security.tokens.PasswordToken;
 +import org.apache.accumulo.core.conf.Property;
 +import org.apache.accumulo.core.data.Key;
 +import org.apache.accumulo.core.data.Mutation;
 +import org.apache.accumulo.core.data.PartialKey;
 +import org.apache.accumulo.core.data.Value;
 +import org.apache.accumulo.core.metadata.MetadataTable;
 +import org.apache.accumulo.core.metadata.schema.MetadataSchema.ReplicationSection;
 +import org.apache.accumulo.core.protobuf.ProtobufUtil;
 +import org.apache.accumulo.core.replication.ReplicationSchema.WorkSection;
 +import org.apache.accumulo.core.replication.ReplicationTable;
 +import org.apache.accumulo.core.security.Authorizations;
 +import org.apache.accumulo.core.security.TablePermission;
 +import org.apache.accumulo.master.replication.SequentialWorkAssigner;
 +import org.apache.accumulo.minicluster.ServerType;
 +import org.apache.accumulo.minicluster.impl.MiniAccumuloClusterImpl;
 +import org.apache.accumulo.minicluster.impl.MiniAccumuloConfigImpl;
 +import org.apache.accumulo.minicluster.impl.ProcessReference;
 +import org.apache.accumulo.server.replication.ReplicaSystemFactory;
 +import org.apache.accumulo.server.replication.StatusUtil;
 +import org.apache.accumulo.server.replication.proto.Replication.Status;
 +import org.apache.accumulo.test.functional.ConfigurableMacBase;
 +import org.apache.accumulo.tserver.TabletServer;
 +import org.apache.accumulo.tserver.replication.AccumuloReplicaSystem;
 +import org.apache.hadoop.conf.Configuration;
 +import org.apache.hadoop.fs.RawLocalFileSystem;
 +import org.junit.After;
 +import org.junit.Assert;
 +import org.junit.Before;
 +import org.junit.Test;
 +import org.slf4j.Logger;
 +import org.slf4j.LoggerFactory;
 +
 +import com.google.common.collect.Iterators;
- import static com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly;
 +
 +/**
 + * Replication tests which start at least two MAC instances and replicate data between them
 + */
 +public class MultiInstanceReplicationIT extends ConfigurableMacBase {
 +  private static final Logger log = LoggerFactory.getLogger(MultiInstanceReplicationIT.class);
 +
 +  private ExecutorService executor;
 +
 +  @Override
 +  public int defaultTimeoutSeconds() {
 +    return 10 * 60;
 +  }
 +
 +  @Before
 +  public void createExecutor() {
 +    executor = Executors.newSingleThreadExecutor();
 +  }
 +
 +  @After
 +  public void stopExecutor() {
 +    if (null != executor) {
 +      executor.shutdownNow();
 +    }
 +  }
 +
 +  @Override
 +  public void configure(MiniAccumuloConfigImpl cfg, Configuration hadoopCoreSite) {
 +    cfg.setNumTservers(1);
 +    cfg.setProperty(Property.INSTANCE_ZK_TIMEOUT, "15s");
 +    cfg.setProperty(Property.TSERV_WALOG_MAX_SIZE, "2M");
 +    cfg.setProperty(Property.GC_CYCLE_START, "1s");
 +    cfg.setProperty(Property.GC_CYCLE_DELAY, "5s");
 +    cfg.setProperty(Property.REPLICATION_WORK_ASSIGNMENT_SLEEP, "1s");
 +    cfg.setProperty(Property.MASTER_REPLICATION_SCAN_INTERVAL, "1s");
 +    cfg.setProperty(Property.REPLICATION_MAX_UNIT_SIZE, "8M");
 +    cfg.setProperty(Property.REPLICATION_NAME, "master");
 +    cfg.setProperty(Property.REPLICATION_WORK_ASSIGNER, SequentialWorkAssigner.class.getName());
 +    cfg.setProperty(Property.TSERV_TOTAL_MUTATION_QUEUE_MAX, "1M");
 +    hadoopCoreSite.set("fs.file.impl", RawLocalFileSystem.class.getName());
 +  }
 +
 +  /**
 +   * Use the same SSL and credential provider configuration that is set up by AbstractMacIT for the other MAC used for replication
 +   */
 +  private void updatePeerConfigFromPrimary(MiniAccumuloConfigImpl primaryCfg, MiniAccumuloConfigImpl peerCfg) {
 +    // Set the same SSL information from the primary when present
 +    Map<String,String> primarySiteConfig = primaryCfg.getSiteConfig();
 +    if ("true".equals(primarySiteConfig.get(Property.INSTANCE_RPC_SSL_ENABLED.getKey()))) {
 +      Map<String,String> peerSiteConfig = new HashMap<>();
 +      peerSiteConfig.put(Property.INSTANCE_RPC_SSL_ENABLED.getKey(), "true");
 +      String keystorePath = primarySiteConfig.get(Property.RPC_SSL_KEYSTORE_PATH.getKey());
 +      Assert.assertNotNull("Keystore Path was null", keystorePath);
 +      peerSiteConfig.put(Property.RPC_SSL_KEYSTORE_PATH.getKey(), keystorePath);
 +      String truststorePath = primarySiteConfig.get(Property.RPC_SSL_TRUSTSTORE_PATH.getKey());
 +      Assert.assertNotNull("Truststore Path was null", truststorePath);
 +      peerSiteConfig.put(Property.RPC_SSL_TRUSTSTORE_PATH.getKey(), truststorePath);
 +
 +      // Passwords might be stored in CredentialProvider
 +      String keystorePassword = primarySiteConfig.get(Property.RPC_SSL_KEYSTORE_PASSWORD.getKey());
 +      if (null != keystorePassword) {
 +        peerSiteConfig.put(Property.RPC_SSL_KEYSTORE_PASSWORD.getKey(), keystorePassword);
 +      }
 +      String truststorePassword = primarySiteConfig.get(Property.RPC_SSL_TRUSTSTORE_PASSWORD.getKey());
 +      if (null != truststorePassword) {
 +        peerSiteConfig.put(Property.RPC_SSL_TRUSTSTORE_PASSWORD.getKey(), truststorePassword);
 +      }
 +
 +      System.out.println("Setting site configuration for peer " + peerSiteConfig);
 +      peerCfg.setSiteConfig(peerSiteConfig);
 +    }
 +
 +    // Use the CredentialProvider if the primary also uses one
 +    String credProvider = primarySiteConfig.get(Property.GENERAL_SECURITY_CREDENTIAL_PROVIDER_PATHS.getKey());
 +    if (null != credProvider) {
 +      Map<String,String> peerSiteConfig = peerCfg.getSiteConfig();
 +      peerSiteConfig.put(Property.GENERAL_SECURITY_CREDENTIAL_PROVIDER_PATHS.getKey(), credProvider);
 +      peerCfg.setSiteConfig(peerSiteConfig);
 +    }
 +  }
 +
 +  @Test(timeout = 10 * 60 * 1000)
 +  public void dataWasReplicatedToThePeer() throws Exception {
 +    MiniAccumuloConfigImpl peerCfg = new MiniAccumuloConfigImpl(createTestDir(this.getClass().getName() + "_" + this.testName.getMethodName() + "_peer"),
 +        ROOT_PASSWORD);
 +    peerCfg.setNumTservers(1);
 +    peerCfg.setInstanceName("peer");
 +    peerCfg.setProperty(Property.REPLICATION_NAME, "peer");
 +
 +    updatePeerConfigFromPrimary(getCluster().getConfig(), peerCfg);
 +
 +    MiniAccumuloClusterImpl peerCluster = new MiniAccumuloClusterImpl(peerCfg);
 +
 +    peerCluster.start();
 +
 +    try {
 +      final Connector connMaster = getConnector();
 +      final Connector connPeer = peerCluster.getConnector("root", new PasswordToken(ROOT_PASSWORD));
 +
 +      ReplicationTable.setOnline(connMaster);
 +
 +      String peerUserName = "peer", peerPassword = "foo";
 +
 +      String peerClusterName = "peer";
 +
 +      connPeer.securityOperations().createLocalUser(peerUserName, new PasswordToken(peerPassword));
 +
 +      connMaster.instanceOperations().setProperty(Property.REPLICATION_PEER_USER.getKey() + peerClusterName, peerUserName);
 +      connMaster.instanceOperations().setProperty(Property.REPLICATION_PEER_PASSWORD.getKey() + peerClusterName, peerPassword);
 +
 +      // ...peer = AccumuloReplicaSystem,instanceName,zookeepers
 +      connMaster.instanceOperations().setProperty(
 +          Property.REPLICATION_PEERS.getKey() + peerClusterName,
 +          ReplicaSystemFactory.getPeerConfigurationValue(AccumuloReplicaSystem.class,
 +              AccumuloReplicaSystem.buildConfiguration(peerCluster.getInstanceName(), peerCluster.getZooKeepers())));
 +
 +      final String masterTable = "master", peerTable = "peer";
 +
 +      connMaster.tableOperations().create(masterTable);
 +      String masterTableId = connMaster.tableOperations().tableIdMap().get(masterTable);
 +      Assert.assertNotNull(masterTableId);
 +
 +      connPeer.tableOperations().create(peerTable);
 +      String peerTableId = connPeer.tableOperations().tableIdMap().get(peerTable);
 +      Assert.assertNotNull(peerTableId);
 +
 +      connPeer.securityOperations().grantTablePermission(peerUserName, peerTable, TablePermission.WRITE);
 +
 +      // Replicate this table to the peerClusterName in a table with the peerTableId table id
 +      connMaster.tableOperations().setProperty(masterTable, Property.TABLE_REPLICATION.getKey(), "true");
 +      connMaster.tableOperations().setProperty(masterTable, Property.TABLE_REPLICATION_TARGET.getKey() + peerClusterName, peerTableId);
 +
 +      // Write some data to table1
 +      BatchWriter bw = connMaster.createBatchWriter(masterTable, new BatchWriterConfig());
 +      for (int rows = 0; rows < 5000; rows++) {
 +        Mutation m = new Mutation(Integer.toString(rows));
 +        for (int cols = 0; cols < 100; cols++) {
 +          String value = Integer.toString(cols);
 +          m.put(value, "", value);
 +        }
 +        bw.addMutation(m);
 +      }
 +
 +      bw.close();
 +
 +      log.info("Wrote all data to master cluster");
 +
 +      final Set<String> filesNeedingReplication = connMaster.replicationOperations().referencedFiles(masterTable);
 +
 +      log.info("Files to replicate: " + filesNeedingReplication);
 +
 +      for (ProcessReference proc : cluster.getProcesses().get(ServerType.TABLET_SERVER)) {
 +        cluster.killProcess(ServerType.TABLET_SERVER, proc);
 +      }
 +      cluster.exec(TabletServer.class);
 +
 +      log.info("TabletServer restarted");
 +      Iterators.size(ReplicationTable.getScanner(connMaster).iterator());
 +      log.info("TabletServer is online");
 +
 +      while (!ReplicationTable.isOnline(connMaster)) {
 +        log.info("Replication table still offline, waiting");
 +        Thread.sleep(5000);
 +      }
 +
 +      log.info("");
 +      log.info("Fetching metadata records:");
 +      for (Entry<Key,Value> kv : connMaster.createScanner(MetadataTable.NAME, Authorizations.EMPTY)) {
 +        if (ReplicationSection.COLF.equals(kv.getKey().getColumnFamily())) {
 +          log.info(kv.getKey().toStringNoTruncate() + " " + ProtobufUtil.toString(Status.parseFrom(kv.getValue().get())));
 +        } else {
 +          log.info(kv.getKey().toStringNoTruncate() + " " + kv.getValue());
 +        }
 +      }
 +
 +      log.info("");
 +      log.info("Fetching replication records:");
 +      for (Entry<Key,Value> kv : ReplicationTable.getScanner(connMaster)) {
 +        log.info(kv.getKey().toStringNoTruncate() + " " + ProtobufUtil.toString(Status.parseFrom(kv.getValue().get())));
 +      }
 +
 +      Future<Boolean> future = executor.submit(new Callable<Boolean>() {
 +
 +        @Override
 +        public Boolean call() throws Exception {
 +          long then = System.currentTimeMillis();
 +          connMaster.replicationOperations().drain(masterTable, filesNeedingReplication);
 +          long now = System.currentTimeMillis();
 +          log.info("Drain completed in " + (now - then) + "ms");
 +          return true;
 +        }
 +
 +      });
 +
 +      try {
 +        future.get(60, TimeUnit.SECONDS);
 +      } catch (TimeoutException e) {
 +        future.cancel(true);
 +        Assert.fail("Drain did not finish within 60 seconds");
 +      } finally {
 +        executor.shutdownNow();
 +      }
 +
 +      log.info("drain completed");
 +
 +      log.info("");
 +      log.info("Fetching metadata records:");
 +      for (Entry<Key,Value> kv : connMaster.createScanner(MetadataTable.NAME, Authorizations.EMPTY)) {
 +        if (ReplicationSection.COLF.equals(kv.getKey().getColumnFamily())) {
 +          log.info(kv.getKey().toStringNoTruncate() + " " + ProtobufUtil.toString(Status.parseFrom(kv.getValue().get())));
 +        } else {
 +          log.info(kv.getKey().toStringNoTruncate() + " " + kv.getValue());
 +        }
 +      }
 +
 +      log.info("");
 +      log.info("Fetching replication records:");
 +      for (Entry<Key,Value> kv : ReplicationTable.getScanner(connMaster)) {
 +        log.info(kv.getKey().toStringNoTruncate() + " " + ProtobufUtil.toString(Status.parseFrom(kv.getValue().get())));
 +      }
 +
 +      Scanner master = connMaster.createScanner(masterTable, Authorizations.EMPTY), peer = connPeer.createScanner(peerTable, Authorizations.EMPTY);
 +      Iterator<Entry<Key,Value>> masterIter = master.iterator(), peerIter = peer.iterator();
 +      Entry<Key,Value> masterEntry = null, peerEntry = null;
 +      while (masterIter.hasNext() && peerIter.hasNext()) {
 +        masterEntry = masterIter.next();
 +        peerEntry = peerIter.next();
 +        Assert.assertEquals(masterEntry.getKey() + " was not equal to " + peerEntry.getKey(), 0,
 +            masterEntry.getKey().compareTo(peerEntry.getKey(), PartialKey.ROW_COLFAM_COLQUAL_COLVIS));
 +        Assert.assertEquals(masterEntry.getValue(), peerEntry.getValue());
 +      }
 +
 +      log.info("Last master entry: " + masterEntry);
 +      log.info("Last peer entry: " + peerEntry);
 +
 +      Assert.assertFalse("Had more data to read from the master", masterIter.hasNext());
 +      Assert.assertFalse("Had more data to read from the peer", peerIter.hasNext());
 +    } finally {
 +      peerCluster.stop();
 +    }
 +  }
 +
 +  @Test
 +  public void dataReplicatedToCorrectTable() throws Exception {
 +    MiniAccumuloConfigImpl peerCfg = new MiniAccumuloConfigImpl(createTestDir(this.getClass().getName() + "_" + this.testName.getMethodName() + "_peer"),
 +        ROOT_PASSWORD);
 +    peerCfg.setNumTservers(1);
 +    peerCfg.setInstanceName("peer");
 +    peerCfg.setProperty(Property.REPLICATION_NAME, "peer");
 +
 +    updatePeerConfigFromPrimary(getCluster().getConfig(), peerCfg);
 +
 +    MiniAccumuloClusterImpl peer1Cluster = new MiniAccumuloClusterImpl(peerCfg);
 +
 +    peer1Cluster.start();
 +
 +    try {
 +      Connector connMaster = getConnector();
 +      Connector connPeer = peer1Cluster.getConnector("root", new PasswordToken(ROOT_PASSWORD));
 +
 +      String peerClusterName = "peer";
 +      String peerUserName = "peer", peerPassword = "foo";
 +
 +      // Create local user
 +      connPeer.securityOperations().createLocalUser(peerUserName, new PasswordToken(peerPassword));
 +
 +      connMaster.instanceOperations().setProperty(Property.REPLICATION_PEER_USER.getKey() + peerClusterName, peerUserName);
 +      connMaster.instanceOperations().setProperty(Property.REPLICATION_PEER_PASSWORD.getKey() + peerClusterName, peerPassword);
 +
 +      // ...peer = AccumuloReplicaSystem,instanceName,zookeepers
 +      connMaster.instanceOperations().setProperty(
 +          Property.REPLICATION_PEERS.getKey() + peerClusterName,
 +          ReplicaSystemFactory.getPeerConfigurationValue(AccumuloReplicaSystem.class,
 +              AccumuloReplicaSystem.buildConfiguration(peer1Cluster.getInstanceName(), peer1Cluster.getZooKeepers())));
 +
 +      String masterTable1 = "master1", peerTable1 = "peer1", masterTable2 = "master2", peerTable2 = "peer2";
 +
 +      // Create tables
 +      connMaster.tableOperations().create(masterTable1);
 +      String masterTableId1 = connMaster.tableOperations().tableIdMap().get(masterTable1);
 +      Assert.assertNotNull(masterTableId1);
 +
 +      connMaster.tableOperations().create(masterTable2);
 +      String masterTableId2 = connMaster.tableOperations().tableIdMap().get(masterTable2);
 +      Assert.assertNotNull(masterTableId2);
 +
 +      connPeer.tableOperations().create(peerTable1);
 +      String peerTableId1 = connPeer.tableOperations().tableIdMap().get(peerTable1);
 +      Assert.assertNotNull(peerTableId1);
 +
 +      connPeer.tableOperations().create(peerTable2);
 +      String peerTableId2 = connPeer.tableOperations().tableIdMap().get(peerTable2);
 +      Assert.assertNotNull(peerTableId2);
 +
 +      // Grant write permission
 +      connPeer.securityOperations().grantTablePermission(peerUserName, peerTable1, TablePermission.WRITE);
 +      connPeer.securityOperations().grantTablePermission(peerUserName, peerTable2, TablePermission.WRITE);
 +
 +      // Replicate this table to the peerClusterName in a table with the peerTableId table id
 +      connMaster.tableOperations().setProperty(masterTable1, Property.TABLE_REPLICATION.getKey(), "true");
 +      connMaster.tableOperations().setProperty(masterTable1, Property.TABLE_REPLICATION_TARGET.getKey() + peerClusterName, peerTableId1);
 +
 +      connMaster.tableOperations().setProperty(masterTable2, Property.TABLE_REPLICATION.getKey(), "true");
 +      connMaster.tableOperations().setProperty(masterTable2, Property.TABLE_REPLICATION_TARGET.getKey() + peerClusterName, peerTableId2);
 +
 +      // Write some data to table1
 +      BatchWriter bw = connMaster.createBatchWriter(masterTable1, new BatchWriterConfig());
 +      long masterTable1Records = 0l;
 +      for (int rows = 0; rows < 2500; rows++) {
 +        Mutation m = new Mutation(masterTable1 + rows);
 +        for (int cols = 0; cols < 100; cols++) {
 +          String value = Integer.toString(cols);
 +          m.put(value, "", value);
 +          masterTable1Records++;
 +        }
 +        bw.addMutation(m);
 +      }
 +
 +      bw.close();
 +
 +      // Write some data to table2
 +      bw = connMaster.createBatchWriter(masterTable2, new BatchWriterConfig());
 +      long masterTable2Records = 0l;
 +      for (int rows = 0; rows < 2500; rows++) {
 +        Mutation m = new Mutation(masterTable2 + rows);
 +        for (int cols = 0; cols < 100; cols++) {
 +          String value = Integer.toString(cols);
 +          m.put(value, "", value);
 +          masterTable2Records++;
 +        }
 +        bw.addMutation(m);
 +      }
 +
 +      bw.close();
 +
 +      log.info("Wrote all data to master cluster");
 +
 +      Set<String> filesFor1 = connMaster.replicationOperations().referencedFiles(masterTable1), filesFor2 = connMaster.replicationOperations().referencedFiles(
 +          masterTable2);
 +
 +      log.info("Files to replicate for table1: " + filesFor1);
 +      log.info("Files to replicate for table2: " + filesFor2);
 +
 +      // Restart the tserver to force a close on the WAL
 +      for (ProcessReference proc : cluster.getProcesses().get(ServerType.TABLET_SERVER)) {
 +        cluster.killProcess(ServerType.TABLET_SERVER, proc);
 +      }
 +      cluster.exec(TabletServer.class);
 +
 +      log.info("Restarted the tserver");
 +
 +      // Read the data -- the tserver is back up and running
 +      Iterators.size(connMaster.createScanner(masterTable1, Authorizations.EMPTY).iterator());
 +
 +      while (!ReplicationTable.isOnline(connMaster)) {
 +        log.info("Replication table still offline, waiting");
 +        Thread.sleep(5000);
 +      }
 +
 +      // Wait for both tables to be replicated
 +      log.info("Waiting for {} for {}", filesFor1, masterTable1);
 +      connMaster.replicationOperations().drain(masterTable1, filesFor1);
 +
 +      log.info("Waiting for {} for {}", filesFor2, masterTable2);
 +      connMaster.replicationOperations().drain(masterTable2, filesFor2);
 +
 +      long countTable = 0l;
 +      for (Entry<Key,Value> entry : connPeer.createScanner(peerTable1, Authorizations.EMPTY)) {
 +        countTable++;
 +        Assert.assertTrue("Found unexpected key-value" + entry.getKey().toStringNoTruncate() + " " + entry.getValue(), entry.getKey().getRow().toString()
 +            .startsWith(masterTable1));
 +      }
 +
 +      log.info("Found {} records in {}", countTable, peerTable1);
 +      Assert.assertEquals(masterTable1Records, countTable);
 +
 +      countTable = 0l;
 +      for (Entry<Key,Value> entry : connPeer.createScanner(peerTable2, Authorizations.EMPTY)) {
 +        countTable++;
 +        Assert.assertTrue("Found unexpected key-value" + entry.getKey().toStringNoTruncate() + " " + entry.getValue(), entry.getKey().getRow().toString()
 +            .startsWith(masterTable2));
 +      }
 +
 +      log.info("Found {} records in {}", countTable, peerTable2);
 +      Assert.assertEquals(masterTable2Records, countTable);
 +
 +    } finally {
 +      peer1Cluster.stop();
 +    }
 +  }
 +
 +  @Test
 +  public void dataWasReplicatedToThePeerWithoutDrain() throws Exception {
 +    MiniAccumuloConfigImpl peerCfg = new MiniAccumuloConfigImpl(createTestDir(this.getClass().getName() + "_" + this.testName.getMethodName() + "_peer"),
 +        ROOT_PASSWORD);
 +    peerCfg.setNumTservers(1);
 +    peerCfg.setInstanceName("peer");
 +    peerCfg.setProperty(Property.REPLICATION_NAME, "peer");
 +
 +    updatePeerConfigFromPrimary(getCluster().getConfig(), peerCfg);
 +
 +    MiniAccumuloClusterImpl peerCluster = new MiniAccumuloClusterImpl(peerCfg);
 +
 +    peerCluster.start();
 +
 +    Connector connMaster = getConnector();
 +    Connector connPeer = peerCluster.getConnector("root", new PasswordToken(ROOT_PASSWORD));
 +
 +    String peerUserName = "repl";
 +    String peerPassword = "passwd";
 +
 +    // Create a user on the peer for replication to use
 +    connPeer.securityOperations().createLocalUser(peerUserName, new PasswordToken(peerPassword));
 +
 +    String peerClusterName = "peer";
 +
 +    // ...peer = AccumuloReplicaSystem,instanceName,zookeepers
 +    connMaster.instanceOperations().setProperty(
 +        Property.REPLICATION_PEERS.getKey() + peerClusterName,
 +        ReplicaSystemFactory.getPeerConfigurationValue(AccumuloReplicaSystem.class,
 +            AccumuloReplicaSystem.buildConfiguration(peerCluster.getInstanceName(), peerCluster.getZooKeepers())));
 +
 +    // Configure the credentials we should use to authenticate ourselves to the peer for replication
 +    connMaster.instanceOperations().setProperty(Property.REPLICATION_PEER_USER.getKey() + peerClusterName, peerUserName);
 +    connMaster.instanceOperations().setProperty(Property.REPLICATION_PEER_PASSWORD.getKey() + peerClusterName, peerPassword);
 +
 +    String masterTable = "master", peerTable = "peer";
 +
 +    connMaster.tableOperations().create(masterTable);
 +    String masterTableId = connMaster.tableOperations().tableIdMap().get(masterTable);
 +    Assert.assertNotNull(masterTableId);
 +
 +    connPeer.tableOperations().create(peerTable);
 +    String peerTableId = connPeer.tableOperations().tableIdMap().get(peerTable);
 +    Assert.assertNotNull(peerTableId);
 +
 +    // Give our replication user the ability to write to the table
 +    connPeer.securityOperations().grantTablePermission(peerUserName, peerTable, TablePermission.WRITE);
 +
 +    // Replicate this table to the peerClusterName in a table with the peerTableId table id
 +    connMaster.tableOperations().setProperty(masterTable, Property.TABLE_REPLICATION.getKey(), "true");
 +    connMaster.tableOperations().setProperty(masterTable, Property.TABLE_REPLICATION_TARGET.getKey() + peerClusterName, peerTableId);
 +
 +    // Write some data to table1
 +    BatchWriter bw = connMaster.createBatchWriter(masterTable, new BatchWriterConfig());
 +    for (int rows = 0; rows < 5000; rows++) {
 +      Mutation m = new Mutation(Integer.toString(rows));
 +      for (int cols = 0; cols < 100; cols++) {
 +        String value = Integer.toString(cols);
 +        m.put(value, "", value);
 +      }
 +      bw.addMutation(m);
 +    }
 +
 +    bw.close();
 +
 +    log.info("Wrote all data to master cluster");
 +
 +    Set<String> files = connMaster.replicationOperations().referencedFiles(masterTable);
 +
 +    log.info("Files to replicate:" + files);
 +
 +    for (ProcessReference proc : cluster.getProcesses().get(ServerType.TABLET_SERVER)) {
 +      cluster.killProcess(ServerType.TABLET_SERVER, proc);
 +    }
 +
 +    cluster.exec(TabletServer.class);
 +
 +    while (!ReplicationTable.isOnline(connMaster)) {
 +      log.info("Replication table still offline, waiting");
 +      Thread.sleep(5000);
 +    }
 +
 +    Iterators.size(connMaster.createScanner(masterTable, Authorizations.EMPTY).iterator());
 +
 +    for (Entry<Key,Value> kv : ReplicationTable.getScanner(connMaster)) {
 +      log.debug(kv.getKey().toStringNoTruncate() + " " + ProtobufUtil.toString(Status.parseFrom(kv.getValue().get())));
 +    }
 +
 +    connMaster.replicationOperations().drain(masterTable, files);
 +
 +    Scanner master = connMaster.createScanner(masterTable, Authorizations.EMPTY), peer = connPeer.createScanner(peerTable, Authorizations.EMPTY);
 +    Iterator<Entry<Key,Value>> masterIter = master.iterator(), peerIter = peer.iterator();
 +    while (masterIter.hasNext() && peerIter.hasNext()) {
 +      Entry<Key,Value> masterEntry = masterIter.next(), peerEntry = peerIter.next();
 +      Assert.assertEquals(peerEntry.getKey() + " was not equal to " + peerEntry.getKey(), 0,
 +          masterEntry.getKey().compareTo(peerEntry.getKey(), PartialKey.ROW_COLFAM_COLQUAL_COLVIS));
 +      Assert.assertEquals(masterEntry.getValue(), peerEntry.getValue());
 +    }
 +
 +    Assert.assertFalse("Had more data to read from the master", masterIter.hasNext());
 +    Assert.assertFalse("Had more data to read from the peer", peerIter.hasNext());
 +
 +    peerCluster.stop();
 +  }
 +
 +  @Test
 +  public void dataReplicatedToCorrectTableWithoutDrain() throws Exception {
 +    MiniAccumuloConfigImpl peerCfg = new MiniAccumuloConfigImpl(createTestDir(this.getClass().getName() + "_" + this.testName.getMethodName() + "_peer"),
 +        ROOT_PASSWORD);
 +    peerCfg.setNumTservers(1);
 +    peerCfg.setInstanceName("peer");
 +    peerCfg.setProperty(Property.REPLICATION_NAME, "peer");
 +
 +    updatePeerConfigFromPrimary(getCluster().getConfig(), peerCfg);
 +
 +    MiniAccumuloClusterImpl peer1Cluster = new MiniAccumuloClusterImpl(peerCfg);
 +
 +    peer1Cluster.start();
 +
 +    try {
 +      Connector connMaster = getConnector();
 +      Connector connPeer = peer1Cluster.getConnector("root", new PasswordToken(ROOT_PASSWORD));
 +
 +      String peerClusterName = "peer";
 +
 +      String peerUserName = "repl";
 +      String peerPassword = "passwd";
 +
 +      // Create a user on the peer for replication to use
 +      connPeer.securityOperations().createLocalUser(peerUserName, new PasswordToken(peerPassword));
 +
 +      // Configure the credentials we should use to authenticate ourselves to the peer for replication
 +      connMaster.instanceOperations().setProperty(Property.REPLICATION_PEER_USER.getKey() + peerClusterName, peerUserName);
 +      connMaster.instanceOperations().setProperty(Property.REPLICATION_PEER_PASSWORD.getKey() + peerClusterName, peerPassword);
 +
 +      // ...peer = AccumuloReplicaSystem,instanceName,zookeepers
 +      connMaster.instanceOperations().setProperty(
 +          Property.REPLICATION_PEERS.getKey() + peerClusterName,
 +          ReplicaSystemFactory.getPeerConfigurationValue(AccumuloReplicaSystem.class,
 +              AccumuloReplicaSystem.buildConfiguration(peer1Cluster.getInstanceName(), peer1Cluster.getZooKeepers())));
 +
 +      String masterTable1 = "master1", peerTable1 = "peer1", masterTable2 = "master2", peerTable2 = "peer2";
 +
 +      connMaster.tableOperations().create(masterTable1);
 +      String masterTableId1 = connMaster.tableOperations().tableIdMap().get(masterTable1);
 +      Assert.assertNotNull(masterTableId1);
 +
 +      connMaster.tableOperations().create(masterTable2);
 +      String masterTableId2 = connMaster.tableOperations().tableIdMap().get(masterTable2);
 +      Assert.assertNotNull(masterTableId2);
 +
 +      connPeer.tableOperations().create(peerTable1);
 +      String peerTableId1 = connPeer.tableOperations().tableIdMap().get(peerTable1);
 +      Assert.assertNotNull(peerTableId1);
 +
 +      connPeer.tableOperations().create(peerTable2);
 +      String peerTableId2 = connPeer.tableOperations().tableIdMap().get(peerTable2);
 +      Assert.assertNotNull(peerTableId2);
 +
 +      // Give our replication user the ability to write to the tables
 +      connPeer.securityOperations().grantTablePermission(peerUserName, peerTable1, TablePermission.WRITE);
 +      connPeer.securityOperations().grantTablePermission(peerUserName, peerTable2, TablePermission.WRITE);
 +
 +      // Replicate this table to the peerClusterName in a table with the peerTableId table id
 +      connMaster.tableOperations().setProperty(masterTable1, Property.TABLE_REPLICATION.getKey(), "true");
 +      connMaster.tableOperations().setProperty(masterTable1, Property.TABLE_REPLICATION_TARGET.getKey() + peerClusterName, peerTableId1);
 +
 +      connMaster.tableOperations().setProperty(masterTable2, Property.TABLE_REPLICATION.getKey(), "true");
 +      connMaster.tableOperations().setProperty(masterTable2, Property.TABLE_REPLICATION_TARGET.getKey() + peerClusterName, peerTableId2);
 +
 +      // Write some data to table1
 +      BatchWriter bw = connMaster.createBatchWriter(masterTable1, new BatchWriterConfig());
 +      for (int rows = 0; rows < 2500; rows++) {
 +        Mutation m = new Mutation(masterTable1 + rows);
 +        for (int cols = 0; cols < 100; cols++) {
 +          String value = Integer.toString(cols);
 +          m.put(value, "", value);
 +        }
 +        bw.addMutation(m);
 +      }
 +
 +      bw.close();
 +
 +      // Write some data to table2
 +      bw = connMaster.createBatchWriter(masterTable2, new BatchWriterConfig());
 +      for (int rows = 0; rows < 2500; rows++) {
 +        Mutation m = new Mutation(masterTable2 + rows);
 +        for (int cols = 0; cols < 100; cols++) {
 +          String value = Integer.toString(cols);
 +          m.put(value, "", value);
 +        }
 +        bw.addMutation(m);
 +      }
 +
 +      bw.close();
 +
 +      log.info("Wrote all data to master cluster");
 +
 +      for (ProcessReference proc : cluster.getProcesses().get(ServerType.TABLET_SERVER)) {
 +        cluster.killProcess(ServerType.TABLET_SERVER, proc);
 +      }
 +
 +      cluster.exec(TabletServer.class);
 +
 +      while (!ReplicationTable.isOnline(connMaster)) {
 +        log.info("Replication table still offline, waiting");
 +        Thread.sleep(5000);
 +      }
 +
 +      // Wait until we fully replicated something
 +      boolean fullyReplicated = false;
 +      for (int i = 0; i < 10 && !fullyReplicated; i++) {
 +        sleepUninterruptibly(2, TimeUnit.SECONDS);
 +
 +        Scanner s = ReplicationTable.getScanner(connMaster);
 +        WorkSection.limit(s);
 +        for (Entry<Key,Value> entry : s) {
 +          Status status = Status.parseFrom(entry.getValue().get());
 +          if (StatusUtil.isFullyReplicated(status)) {
 +            fullyReplicated |= true;
 +          }
 +        }
 +      }
 +
 +      Assert.assertNotEquals(0, fullyReplicated);
 +
 +      // We have to wait for the master to assign the replication work, a local tserver to process it, and then the remote tserver to replay it
 +      // Be cautious in how quickly we assert that the data is present on the peer
 +      long countTable = 0l;
 +      for (int i = 0; i < 10; i++) {
 +        for (Entry<Key,Value> entry : connPeer.createScanner(peerTable1, Authorizations.EMPTY)) {
 +          countTable++;
 +          Assert.assertTrue("Found unexpected key-value" + entry.getKey().toStringNoTruncate() + " " + entry.getValue(), entry.getKey().getRow().toString()
 +              .startsWith(masterTable1));
 +        }
 +
 +        log.info("Found {} records in {}", countTable, peerTable1);
 +
 +        if (0l == countTable) {
 +          Thread.sleep(5000);
 +        } else {
 +          break;
 +        }
 +      }
 +
 +      Assert.assertTrue("Found no records in " + peerTable1 + " in the peer cluster", countTable > 0);
 +
 +      // We have to wait for the master to assign the replication work, a local tserver to process it, and then the remote tserver to replay it
 +      // Be cautious in how quickly we assert that the data is present on the peer
 +      for (int i = 0; i < 10; i++) {
 +        countTable = 0l;
 +        for (Entry<Key,Value> entry : connPeer.createScanner(peerTable2, Authorizations.EMPTY)) {
 +          countTable++;
 +          Assert.assertTrue("Found unexpected key-value" + entry.getKey().toStringNoTruncate() + " " + entry.getValue(), entry.getKey().getRow().toString()
 +              .startsWith(masterTable2));
 +        }
 +
 +        log.info("Found {} records in {}", countTable, peerTable2);
 +
 +        if (0l == countTable) {
 +          Thread.sleep(5000);
 +        } else {
 +          break;
 +        }
 +      }
 +
 +      Assert.assertTrue("Found no records in " + peerTable2 + " in the peer cluster", countTable > 0);
 +
 +    } finally {
 +      peer1Cluster.stop();
 +    }
 +  }
 +}


Mime
View raw message