You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by el...@apache.org on 2014/01/23 19:05:50 UTC

[10/15] git commit: Merge branch '1.4.5-SNAPSHOT' into 1.5.1-SNAPSHOT

Merge branch '1.4.5-SNAPSHOT' into 1.5.1-SNAPSHOT

Conflicts:
	src/core/src/main/java/org/apache/accumulo/core/client/mapreduce/InputFormatBase.java
	test/src/main/java/org/apache/accumulo/test/continuous/ContinuousVerify.java


Project: http://git-wip-us.apache.org/repos/asf/accumulo/repo
Commit: http://git-wip-us.apache.org/repos/asf/accumulo/commit/a147acdd
Tree: http://git-wip-us.apache.org/repos/asf/accumulo/tree/a147acdd
Diff: http://git-wip-us.apache.org/repos/asf/accumulo/diff/a147acdd

Branch: refs/heads/1.6.0-SNAPSHOT
Commit: a147acdd600f2fce2a78c9845d00ac3779185707
Parents: 00bd820 9cf94f9
Author: Josh Elser <el...@apache.org>
Authored: Thu Jan 23 12:31:48 2014 -0500
Committer: Josh Elser <el...@apache.org>
Committed: Thu Jan 23 12:31:48 2014 -0500

----------------------------------------------------------------------
 .../core/client/mapreduce/RangeInputSplit.java  |    2 +-
 .../core/client/mapreduce/InputFormatBase.java  | 1634 ++++++++++++++++++
 .../test/continuous/ContinuousVerify.java       |   82 +-
 3 files changed, 1676 insertions(+), 42 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/a147acdd/core/src/main/java/org/apache/accumulo/core/client/mapreduce/RangeInputSplit.java
----------------------------------------------------------------------
diff --cc core/src/main/java/org/apache/accumulo/core/client/mapreduce/RangeInputSplit.java
index 592cde6,0000000..f7b2263
mode 100644,000000..100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/RangeInputSplit.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/RangeInputSplit.java
@@@ -1,442 -1,0 +1,442 @@@
 +/*
 + * 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.core.client.mapreduce;
 +
 +import java.io.DataInput;
 +import java.io.DataOutput;
 +import java.io.IOException;
 +import java.math.BigInteger;
 +import java.nio.charset.Charset;
 +import java.util.ArrayList;
 +import java.util.Arrays;
 +import java.util.List;
 +import java.util.Set;
 +
 +import org.apache.accumulo.core.client.AccumuloSecurityException;
 +import org.apache.accumulo.core.client.Instance;
 +import org.apache.accumulo.core.client.IteratorSetting;
 +import org.apache.accumulo.core.client.ZooKeeperInstance;
 +import org.apache.accumulo.core.client.mapreduce.lib.util.InputConfigurator;
 +import org.apache.accumulo.core.client.mock.MockInstance;
 +import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
 +import org.apache.accumulo.core.conf.AccumuloConfiguration;
 +import org.apache.accumulo.core.conf.SiteConfiguration;
 +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.security.Authorizations;
 +import org.apache.accumulo.core.security.CredentialHelper;
 +import org.apache.accumulo.core.util.Pair;
 +import org.apache.commons.codec.binary.Base64;
 +import org.apache.hadoop.io.Text;
 +import org.apache.hadoop.io.Writable;
 +import org.apache.hadoop.mapreduce.InputSplit;
 +import org.apache.log4j.Level;
 +
 +/**
 + * The Class RangeInputSplit. Encapsulates an Accumulo range for use in Map Reduce jobs.
 + */
 +public class RangeInputSplit extends InputSplit implements Writable {
 +  private Range range;
 +  private String[] locations;
 +  private String table, instanceName, zooKeepers, principal;
 +  private AuthenticationToken token;
 +  private Boolean offline, mockInstance, isolatedScan, localIterators;
 +  private Authorizations auths;
 +  private Set<Pair<Text,Text>> fetchedColumns;
 +  private List<IteratorSetting> iterators;
 +  private Level level;
 +
 +  public RangeInputSplit() {
 +    range = new Range();
 +    locations = new String[0];
 +  }
 +
 +  public RangeInputSplit(Range range, String[] locations) {
 +    this.range = range;
 +    this.locations = locations;
 +  }
 +
 +  public Range getRange() {
 +    return range;
 +  }
 +
 +  private static byte[] extractBytes(ByteSequence seq, int numBytes) {
 +    byte[] bytes = new byte[numBytes + 1];
 +    bytes[0] = 0;
 +    for (int i = 0; i < numBytes; i++) {
 +      if (i >= seq.length())
 +        bytes[i + 1] = 0;
 +      else
 +        bytes[i + 1] = seq.byteAt(i);
 +    }
 +    return bytes;
 +  }
 +
 +  public static float getProgress(ByteSequence start, ByteSequence end, ByteSequence position) {
 +    int maxDepth = Math.min(Math.max(end.length(), start.length()), position.length());
 +    BigInteger startBI = new BigInteger(extractBytes(start, maxDepth));
 +    BigInteger endBI = new BigInteger(extractBytes(end, maxDepth));
 +    BigInteger positionBI = new BigInteger(extractBytes(position, maxDepth));
 +    return (float) (positionBI.subtract(startBI).doubleValue() / endBI.subtract(startBI).doubleValue());
 +  }
 +
 +  public float getProgress(Key currentKey) {
 +    if (currentKey == null)
 +      return 0f;
 +    if (range.getStartKey() != null && range.getEndKey() != null) {
 +      if (range.getStartKey().compareTo(range.getEndKey(), PartialKey.ROW) != 0) {
 +        // just look at the row progress
 +        return getProgress(range.getStartKey().getRowData(), range.getEndKey().getRowData(), currentKey.getRowData());
 +      } else if (range.getStartKey().compareTo(range.getEndKey(), PartialKey.ROW_COLFAM) != 0) {
 +        // just look at the column family progress
 +        return getProgress(range.getStartKey().getColumnFamilyData(), range.getEndKey().getColumnFamilyData(), currentKey.getColumnFamilyData());
 +      } else if (range.getStartKey().compareTo(range.getEndKey(), PartialKey.ROW_COLFAM_COLQUAL) != 0) {
 +        // just look at the column qualifier progress
 +        return getProgress(range.getStartKey().getColumnQualifierData(), range.getEndKey().getColumnQualifierData(), currentKey.getColumnQualifierData());
 +      }
 +    }
 +    // if we can't figure it out, then claim no progress
 +    return 0f;
 +  }
 +
 +  /**
 +   * This implementation of length is only an estimate, it does not provide exact values. Do not have your code rely on this return value.
 +   */
 +  @Override
 +  public long getLength() throws IOException {
 +    Text startRow = range.isInfiniteStartKey() ? new Text(new byte[] {Byte.MIN_VALUE}) : range.getStartKey().getRow();
 +    Text stopRow = range.isInfiniteStopKey() ? new Text(new byte[] {Byte.MAX_VALUE}) : range.getEndKey().getRow();
 +    int maxCommon = Math.min(7, Math.min(startRow.getLength(), stopRow.getLength()));
 +    long diff = 0;
 +
 +    byte[] start = startRow.getBytes();
 +    byte[] stop = stopRow.getBytes();
 +    for (int i = 0; i < maxCommon; ++i) {
 +      diff |= 0xff & (start[i] ^ stop[i]);
 +      diff <<= Byte.SIZE;
 +    }
 +
 +    if (startRow.getLength() != stopRow.getLength())
 +      diff |= 0xff;
 +
 +    return diff + 1;
 +  }
 +
 +  @Override
 +  public String[] getLocations() throws IOException {
 +    return locations;
 +  }
 +
 +  @Override
 +  public void readFields(DataInput in) throws IOException {
 +    range.readFields(in);
 +    int numLocs = in.readInt();
 +    locations = new String[numLocs];
 +    for (int i = 0; i < numLocs; ++i)
 +      locations[i] = in.readUTF();
 +
 +    if (in.readBoolean()) {
 +      isolatedScan = in.readBoolean();
 +    }
 +
 +    if (in.readBoolean()) {
 +      offline = in.readBoolean();
 +    }
 +
 +    if (in.readBoolean()) {
 +      localIterators = in.readBoolean();
 +    }
 +
 +    if (in.readBoolean()) {
 +      mockInstance = in.readBoolean();
 +    }
 +
 +    if (in.readBoolean()) {
 +      int numColumns = in.readInt();
 +      List<String> columns = new ArrayList<String>(numColumns);
 +      for (int i = 0; i < numColumns; i++) {
 +        columns.add(in.readUTF());
 +      }
 +
 +      fetchedColumns = InputConfigurator.deserializeFetchedColumns(columns);
 +    }
 +
 +    if (in.readBoolean()) {
 +      String strAuths = in.readUTF();
 +      auths = new Authorizations(strAuths.getBytes(Charset.forName("UTF-8")));
 +    }
 +
 +    if (in.readBoolean()) {
 +      principal = in.readUTF();
 +    }
 +
 +    if (in.readBoolean()) {
 +      String tokenClass = in.readUTF();
 +      byte[] base64TokenBytes = in.readUTF().getBytes(Charset.forName("UTF-8"));
 +      byte[] tokenBytes = Base64.decodeBase64(base64TokenBytes);
 +
 +      try {
 +        token = CredentialHelper.extractToken(tokenClass, tokenBytes);
 +      } catch (AccumuloSecurityException e) {
 +        throw new IOException(e);
 +      }
 +    }
 +
 +    if (in.readBoolean()) {
 +      instanceName = in.readUTF();
 +    }
 +
 +    if (in.readBoolean()) {
 +      zooKeepers = in.readUTF();
 +    }
 +
 +    if (in.readBoolean()) {
 +      level = Level.toLevel(in.readInt());
 +    }
 +  }
 +
 +  @Override
 +  public void write(DataOutput out) throws IOException {
 +    range.write(out);
 +    out.writeInt(locations.length);
 +    for (int i = 0; i < locations.length; ++i)
 +      out.writeUTF(locations[i]);
 +
 +    out.writeBoolean(null != isolatedScan);
 +    if (null != isolatedScan) {
 +      out.writeBoolean(isolatedScan);
 +    }
 +
 +    out.writeBoolean(null != offline);
 +    if (null != offline) {
 +      out.writeBoolean(offline);
 +    }
 +
 +    out.writeBoolean(null != localIterators);
 +    if (null != localIterators) {
 +      out.writeBoolean(localIterators);
 +    }
 +
 +    out.writeBoolean(null != mockInstance);
 +    if (null != mockInstance) {
 +      out.writeBoolean(mockInstance);
 +    }
 +
 +    out.writeBoolean(null != fetchedColumns);
 +    if (null != fetchedColumns) {
 +      String[] cols = InputConfigurator.serializeColumns(fetchedColumns);
 +      out.writeInt(cols.length);
 +      for (String col : cols) {
 +        out.writeUTF(col);
 +      }
 +    }
 +
 +    out.writeBoolean(null != auths);
 +    if (null != auths) {
 +      out.writeUTF(auths.serialize());
 +    }
 +
 +    out.writeBoolean(null != principal);
 +    if (null != principal) {
 +      out.writeUTF(principal);
 +    }
 +
 +    out.writeBoolean(null != token);
 +    if (null != token) {
 +      out.writeUTF(token.getClass().getCanonicalName());
 +      try {
 +        out.writeUTF(CredentialHelper.tokenAsBase64(token));
 +      } catch (AccumuloSecurityException e) {
 +        throw new IOException(e);
 +      }
 +    }
 +
 +    out.writeBoolean(null != instanceName);
 +    if (null != instanceName) {
 +      out.writeUTF(instanceName);
 +    }
 +
 +    out.writeBoolean(null != zooKeepers);
 +    if (null != zooKeepers) {
 +      out.writeUTF(zooKeepers);
 +    }
 +
 +    out.writeBoolean(null != level);
 +    if (null != level) {
 +      out.writeInt(level.toInt());
 +    }
 +  }
 +
 +  @Override
 +  public String toString() {
 +    StringBuilder sb = new StringBuilder(256);
 +    sb.append("Range: ").append(range);
 +    sb.append(" Locations: ").append(Arrays.asList(locations));
 +    sb.append(" Table: ").append(table);
 +    sb.append(" InstanceName: ").append(instanceName);
 +    sb.append(" zooKeepers: ").append(zooKeepers);
 +    sb.append(" principal: ").append(principal);
 +    sb.append(" authenticationToken: ").append(token);
 +    sb.append(" Authorizations: ").append(auths);
 +    sb.append(" offlineScan: ").append(offline);
 +    sb.append(" mockInstance: ").append(mockInstance);
 +    sb.append(" isolatedScan: ").append(isolatedScan);
 +    sb.append(" localIterators: ").append(localIterators);
 +    sb.append(" fetchColumns: ").append(fetchedColumns);
 +    sb.append(" iterators: ").append(iterators);
 +    sb.append(" logLevel: ").append(level);
 +    return sb.toString();
 +  }
 +
 +  public String getTable() {
 +    return table;
 +  }
 +
 +  public void setTable(String table) {
 +    this.table = table;
 +  }
 +
 +  public Instance getInstance() {
 +    if (null == instanceName) {
 +      return null;
 +    }
 +
 +    if (isMockInstance()) {
 +      return new MockInstance(getInstanceName());
 +    }
 +
 +    if (null == zooKeepers) {
 +      return null;
 +    }
-     
++
 +    ZooKeeperInstance zki = new ZooKeeperInstance(getInstanceName(), getZooKeepers());
 +
 +    // Wrap the DefaultConfiguration with a SiteConfiguration so we use accumulo-site.xml
 +    // when it's present
 +    AccumuloConfiguration xmlConfig = SiteConfiguration.getInstance(zki.getConfiguration());
 +    zki.setConfiguration(xmlConfig);
 +
 +    return zki;
 +  }
 +
 +  public String getInstanceName() {
 +    return instanceName;
 +  }
 +
 +  public void setInstanceName(String instanceName) {
 +    this.instanceName = instanceName;
 +  }
 +
 +  public String getZooKeepers() {
 +    return zooKeepers;
 +  }
 +
 +  public void setZooKeepers(String zooKeepers) {
 +    this.zooKeepers = zooKeepers;
 +  }
 +
 +  public String getPrincipal() {
 +    return principal;
 +  }
 +
 +  public void setPrincipal(String principal) {
 +    this.principal = principal;
 +  }
 +
 +  public AuthenticationToken getToken() {
 +    return token;
 +  }
 +
 +  public void setToken(AuthenticationToken token) {
 +    this.token = token;
 +    ;
 +  }
 +
 +  public Boolean isOffline() {
 +    return offline;
 +  }
 +
 +  public void setOffline(Boolean offline) {
 +    this.offline = offline;
 +  }
 +
 +  public void setLocations(String[] locations) {
 +    this.locations = locations;
 +  }
 +
 +  public Boolean isMockInstance() {
 +    return mockInstance;
 +  }
 +
 +  public void setMockInstance(Boolean mockInstance) {
 +    this.mockInstance = mockInstance;
 +  }
 +
 +  public Boolean isIsolatedScan() {
 +    return isolatedScan;
 +  }
 +
 +  public void setIsolatedScan(Boolean isolatedScan) {
 +    this.isolatedScan = isolatedScan;
 +  }
 +
 +  public Authorizations getAuths() {
 +    return auths;
 +  }
 +
 +  public void setAuths(Authorizations auths) {
 +    this.auths = auths;
 +  }
 +
 +  public void setRange(Range range) {
 +    this.range = range;
 +  }
 +
 +  public Boolean usesLocalIterators() {
 +    return localIterators;
 +  }
 +
 +  public void setUsesLocalIterators(Boolean localIterators) {
 +    this.localIterators = localIterators;
 +  }
 +
 +  public Set<Pair<Text,Text>> getFetchedColumns() {
 +    return fetchedColumns;
 +  }
 +
 +  public void setFetchedColumns(Set<Pair<Text,Text>> fetchedColumns) {
 +    this.fetchedColumns = fetchedColumns;
 +  }
 +
 +  public List<IteratorSetting> getIterators() {
 +    return iterators;
 +  }
 +
 +  public void setIterators(List<IteratorSetting> iterators) {
 +    this.iterators = iterators;
 +  }
 +
 +  public Level getLogLevel() {
 +    return level;
 +  }
 +
 +  public void setLogLevel(Level level) {
 +    this.level = level;
 +  }
 +}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a147acdd/test/src/main/java/org/apache/accumulo/test/continuous/ContinuousVerify.java
----------------------------------------------------------------------
diff --cc test/src/main/java/org/apache/accumulo/test/continuous/ContinuousVerify.java
index 69a483f,0000000..a9d7153
mode 100644,000000..100644
--- a/test/src/main/java/org/apache/accumulo/test/continuous/ContinuousVerify.java
+++ b/test/src/main/java/org/apache/accumulo/test/continuous/ContinuousVerify.java
@@@ -1,265 -1,0 +1,265 @@@
 +/*
 + * 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.continuous;
 +
 +import java.io.IOException;
 +import java.lang.reflect.Method;
 +import java.net.URI;
 +import java.util.ArrayList;
 +import java.util.HashMap;
 +import java.util.HashSet;
 +import java.util.Random;
 +import java.util.Set;
 +
 +import org.apache.accumulo.core.cli.ClientOnDefaultTable;
 +import org.apache.accumulo.core.client.Connector;
 +import org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat;
 +import org.apache.accumulo.core.data.Key;
 +import org.apache.accumulo.core.data.Range;
 +import org.apache.accumulo.core.data.Value;
 +import org.apache.accumulo.core.util.CachedConfiguration;
 +import org.apache.accumulo.test.continuous.ContinuousWalk.BadChecksumException;
 +import org.apache.hadoop.conf.Configured;
 +import org.apache.hadoop.filecache.DistributedCache;
 +import org.apache.hadoop.fs.FileSystem;
 +import org.apache.hadoop.fs.Path;
 +import org.apache.hadoop.io.LongWritable;
 +import org.apache.hadoop.io.Text;
 +import org.apache.hadoop.io.VLongWritable;
 +import org.apache.hadoop.mapred.Counters.Counter;
 +import org.apache.hadoop.mapreduce.Job;
 +import org.apache.hadoop.mapreduce.Mapper;
 +import org.apache.hadoop.mapreduce.Reducer;
 +import org.apache.hadoop.mapreduce.lib.output.TextOutputFormat;
 +import org.apache.hadoop.util.Tool;
 +import org.apache.hadoop.util.ToolRunner;
 +import org.apache.log4j.Logger;
 +
 +import com.beust.jcommander.Parameter;
 +import com.beust.jcommander.validators.PositiveInteger;
 +
 +/**
 + * A map reduce job that verifies a table created by continuous ingest. It verifies that all referenced nodes are defined.
 + */
 +
 +public class ContinuousVerify extends Configured implements Tool {
 +  private static final Logger log = Logger.getLogger(ContinuousVerify.class);
-   
++
 +  // work around hadoop-1/hadoop-2 runtime incompatibility
 +  static private Method INCREMENT;
 +  static {
 +    try {
 +      INCREMENT = Counter.class.getMethod("increment", Long.TYPE);
 +    } catch (Exception ex) {
 +      throw new RuntimeException(ex);
 +    }
 +  }
-   
++
 +  static void increment(Object obj) {
 +    try {
 +      INCREMENT.invoke(obj, 1L);
 +    } catch (Exception ex) {
 +      throw new RuntimeException(ex);
 +    }
 +  }
-   
++
 +  public static final VLongWritable DEF = new VLongWritable(-1);
-   
++
 +  public static class CMapper extends Mapper<Key,Value,LongWritable,VLongWritable> {
-     
++
 +    private LongWritable row = new LongWritable();
 +    private LongWritable ref = new LongWritable();
 +    private VLongWritable vrow = new VLongWritable();
 +
 +    private long corrupt = 0;
-     
++
 +    @Override
 +    public void map(Key key, Value data, Context context) throws IOException, InterruptedException {
 +      long r = Long.parseLong(key.getRow().toString(), 16);
 +      if (r < 0)
 +        throw new IllegalArgumentException();
-       
++
 +      try {
 +        ContinuousWalk.validate(key, data);
 +      } catch (BadChecksumException bce) {
 +        increment(context.getCounter(Counts.CORRUPT));
 +        if (corrupt < 1000) {
 +          System.out.println("ERROR Bad checksum : " + key);
 +        } else if (corrupt == 1000) {
 +          System.out.println("Too many bad checksums, not printing anymore!");
 +        }
 +        corrupt++;
 +        return;
 +      }
-       
++
 +      row.set(r);
-       
++
 +      context.write(row, DEF);
 +      byte[] val = data.get();
-       
++
 +      int offset = ContinuousWalk.getPrevRowOffset(val);
 +      if (offset > 0) {
 +        ref.set(Long.parseLong(new String(val, offset, 16), 16));
 +        vrow.set(r);
 +        context.write(ref, vrow);
 +      }
 +    }
 +  }
-   
++
 +  public static enum Counts {
 +    UNREFERENCED, UNDEFINED, REFERENCED, CORRUPT
 +  }
-   
++
 +  public static class CReducer extends Reducer<LongWritable,VLongWritable,Text,Text> {
 +    private ArrayList<Long> refs = new ArrayList<Long>();
-     
++
 +    @Override
 +    public void reduce(LongWritable key, Iterable<VLongWritable> values, Context context) throws IOException, InterruptedException {
-       
++
 +      int defCount = 0;
-       
++
 +      refs.clear();
 +      for (VLongWritable type : values) {
 +        if (type.get() == -1) {
 +          defCount++;
 +        } else {
 +          refs.add(type.get());
 +        }
 +      }
-       
++
 +      if (defCount == 0 && refs.size() > 0) {
 +        StringBuilder sb = new StringBuilder();
 +        String comma = "";
 +        for (Long ref : refs) {
 +          sb.append(comma);
 +          comma = ",";
 +          sb.append(new String(ContinuousIngest.genRow(ref)));
 +        }
-         
++
 +        context.write(new Text(ContinuousIngest.genRow(key.get())), new Text(sb.toString()));
 +        increment(context.getCounter(Counts.UNDEFINED));
-         
++
 +      } else if (defCount > 0 && refs.size() == 0) {
 +        increment(context.getCounter(Counts.UNREFERENCED));
 +      } else {
 +        increment(context.getCounter(Counts.REFERENCED));
 +      }
-       
++
 +    }
 +  }
-   
++
 +  static class Opts extends ClientOnDefaultTable {
 +    @Parameter(names = "--output", description = "location in HDFS to store the results; must not exist", required = true)
 +    String outputDir = "/tmp/continuousVerify";
-     
++
 +    @Parameter(names = "--maxMappers", description = "the maximum number of mappers to use", required = true, validateWith = PositiveInteger.class)
 +    int maxMaps = 0;
-     
++
 +    @Parameter(names = "--reducers", description = "the number of reducers to use", required = true, validateWith = PositiveInteger.class)
 +    int reducers = 0;
-     
++
 +    @Parameter(names = "--offline", description = "perform the verification directly on the files while the table is offline")
 +    boolean scanOffline = false;
-     
++
 +    @Parameter(names = "--sitefile", description = "location of accumulo-site.xml in HDFS", required = true)
 +    String siteFile;
-     
++
 +    public Opts() {
 +      super("ci");
 +    }
 +  }
-   
++
 +  @Override
 +  public int run(String[] args) throws Exception {
 +    Opts opts = new Opts();
 +    opts.parseArgs(this.getClass().getName(), args);
-     
++
 +    Job job = new Job(getConf(), this.getClass().getSimpleName() + "_" + System.currentTimeMillis());
 +    job.setJarByClass(this.getClass());
-     
++
 +    job.setInputFormatClass(AccumuloInputFormat.class);
 +    opts.setAccumuloConfigs(job);
 +
 +    String clone = opts.getTableName();
 +    Connector conn = null;
 +    if (opts.scanOffline) {
 +      Random random = new Random();
 +      clone = opts.getTableName() + "_" + String.format("%016x", (random.nextLong() & 0x7fffffffffffffffl));
 +      conn = opts.getConnector();
 +      conn.tableOperations().clone(opts.getTableName(), clone, true, new HashMap<String,String>(), new HashSet<String>());
 +      conn.tableOperations().offline(clone);
 +      AccumuloInputFormat.setInputTableName(job, clone);
 +      AccumuloInputFormat.setOfflineTableScan(job, true);
 +    }
-     
++
 +    // set up ranges
 +    try {
 +      Set<Range> ranges = opts.getConnector().tableOperations().splitRangeByTablets(opts.getTableName(), new Range(), opts.maxMaps);
 +      AccumuloInputFormat.setRanges(job, ranges);
 +      AccumuloInputFormat.setAutoAdjustRanges(job, false);
 +    } catch (Exception e) {
 +      throw new IOException(e);
 +    }
-     
++
 +    job.setMapperClass(CMapper.class);
 +    job.setMapOutputKeyClass(LongWritable.class);
 +    job.setMapOutputValueClass(VLongWritable.class);
-     
++
 +    job.setReducerClass(CReducer.class);
 +    job.setNumReduceTasks(opts.reducers);
-     
++
 +    job.setOutputFormatClass(TextOutputFormat.class);
-     
++
 +    job.getConfiguration().setBoolean("mapred.map.tasks.speculative.execution", opts.scanOffline);
-     
++
 +    TextOutputFormat.setOutputPath(job, new Path(opts.outputDir));
-     
++
 +    Path sitePath = new Path(opts.siteFile);
 +    Path siteParentPath = sitePath.getParent();
 +    if (null == siteParentPath) {
 +      siteParentPath = new Path("/");
 +    }
-     
++
 +    URI siteUri = new URI("hdfs://" + opts.siteFile);
-     
++
 +    log.info("Adding " + siteUri + " to DistributedCache");
-     
++
 +    // Make sure that accumulo-site.xml is available for mappers running offline scans
 +    // as they need to correctly choose instance.dfs.dir for the installation
 +    DistributedCache.addFileToClassPath(siteParentPath, job.getConfiguration(), FileSystem.get(siteUri, job.getConfiguration()));
-     
++
 +    job.waitForCompletion(true);
-     
++
 +    if (opts.scanOffline) {
 +      conn.tableOperations().delete(clone);
 +    }
 +    opts.stopTracing();
 +    return job.isSuccessful() ? 0 : 1;
 +  }
-   
++
 +  /**
 +   * 
 +   * @param args
 +   *          instanceName zookeepers username password table columns outputpath
 +   * @throws Exception
 +   */
 +  public static void main(String[] args) throws Exception {
 +    int res = ToolRunner.run(CachedConfiguration.getInstance(), new ContinuousVerify(), args);
 +    if (res != 0)
 +      System.exit(res);
 +  }
 +}