Return-Path: X-Original-To: archive-asf-public-internal@cust-asf2.ponee.io Delivered-To: archive-asf-public-internal@cust-asf2.ponee.io Received: from cust-asf.ponee.io (cust-asf.ponee.io [163.172.22.183]) by cust-asf2.ponee.io (Postfix) with ESMTP id 37E5A200B29 for ; Thu, 30 Jun 2016 22:42:03 +0200 (CEST) Received: by cust-asf.ponee.io (Postfix) id 36629160A63; Thu, 30 Jun 2016 20:42:03 +0000 (UTC) Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by cust-asf.ponee.io (Postfix) with SMTP id 15654160A52 for ; Thu, 30 Jun 2016 22:42:00 +0200 (CEST) Received: (qmail 56458 invoked by uid 500); 30 Jun 2016 20:41:58 -0000 Mailing-List: contact commits-help@accumulo.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@accumulo.apache.org Delivered-To: mailing list commits@accumulo.apache.org Received: (qmail 56035 invoked by uid 99); 30 Jun 2016 20:41:47 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Thu, 30 Jun 2016 20:41:47 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id 508D1E08FE; Thu, 30 Jun 2016 20:41:44 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: ctubbsii@apache.org To: commits@accumulo.apache.org Date: Thu, 30 Jun 2016 20:41:47 -0000 Message-Id: <9a541d7eb1d943528ec42304a75d3af6@git.apache.org> In-Reply-To: <71d93c73d8e3468a8e3cbe1827be648a@git.apache.org> References: <71d93c73d8e3468a8e3cbe1827be648a@git.apache.org> X-Mailer: ASF-Git Admin Mailer Subject: [4/8] accumulo git commit: ACCUMULO-2589 Drop deprecated code from 1.x archived-at: Thu, 30 Jun 2016 20:42:03 -0000 http://git-wip-us.apache.org/repos/asf/accumulo/blob/b69291a3/core/src/main/java/org/apache/accumulo/core/iterators/aggregation/NumSummation.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/org/apache/accumulo/core/iterators/aggregation/NumSummation.java b/core/src/main/java/org/apache/accumulo/core/iterators/aggregation/NumSummation.java deleted file mode 100644 index 4d79894..0000000 --- a/core/src/main/java/org/apache/accumulo/core/iterators/aggregation/NumSummation.java +++ /dev/null @@ -1,91 +0,0 @@ -/* - * 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.iterators.aggregation; - -import java.io.ByteArrayInputStream; -import java.io.ByteArrayOutputStream; -import java.io.DataInputStream; -import java.io.DataOutputStream; -import java.io.IOException; - -import org.apache.accumulo.core.data.Value; -import org.apache.hadoop.io.WritableUtils; - -/** - * @deprecated since 1.4, replaced by {@link org.apache.accumulo.core.iterators.user.SummingCombiner} with - * {@link org.apache.accumulo.core.iterators.LongCombiner.Type#VARLEN} - */ -@Deprecated -public class NumSummation implements Aggregator { - long sum = 0l; - - @Override - public Value aggregate() { - try { - return new Value(NumSummation.longToBytes(sum)); - } catch (IOException e) { - throw new RuntimeException(e); - } - } - - @Override - public void collect(Value value) { - long l; - try { - l = NumSummation.bytesToLong(value.get()); - } catch (IOException e) { - throw new RuntimeException(e); - } - - sum = NumSummation.safeAdd(sum, l); - } - - public static byte[] longToBytes(long l) throws IOException { - ByteArrayOutputStream baos = new ByteArrayOutputStream(); - DataOutputStream dos = new DataOutputStream(baos); - - WritableUtils.writeVLong(dos, l); - - return baos.toByteArray(); - } - - public static long bytesToLong(byte[] b) throws IOException { - DataInputStream dis = new DataInputStream(new ByteArrayInputStream(b)); - return WritableUtils.readVLong(dis); - } - - public static long safeAdd(long a, long b) { - long aSign = Long.signum(a); - long bSign = Long.signum(b); - if ((aSign != 0) && (bSign != 0) && (aSign == bSign)) { - if (aSign > 0) { - if (Long.MAX_VALUE - a < b) - return Long.MAX_VALUE; - } else { - if (Long.MIN_VALUE - a > b) - return Long.MIN_VALUE; - } - } - return a + b; - } - - @Override - public void reset() { - sum = 0l; - } - -} http://git-wip-us.apache.org/repos/asf/accumulo/blob/b69291a3/core/src/main/java/org/apache/accumulo/core/iterators/aggregation/StringMax.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/org/apache/accumulo/core/iterators/aggregation/StringMax.java b/core/src/main/java/org/apache/accumulo/core/iterators/aggregation/StringMax.java deleted file mode 100644 index 3d4516d..0000000 --- a/core/src/main/java/org/apache/accumulo/core/iterators/aggregation/StringMax.java +++ /dev/null @@ -1,48 +0,0 @@ -/* - * 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.iterators.aggregation; - -import org.apache.accumulo.core.data.Value; - -/** - * @deprecated since 1.4, replaced by {@link org.apache.accumulo.core.iterators.user.MaxCombiner} with - * {@link org.apache.accumulo.core.iterators.LongCombiner.Type#STRING} - */ -@Deprecated -public class StringMax implements Aggregator { - - long max = Long.MIN_VALUE; - - @Override - public Value aggregate() { - return new Value(Long.toString(max).getBytes()); - } - - @Override - public void collect(Value value) { - long l = Long.parseLong(new String(value.get())); - if (l > max) { - max = l; - } - } - - @Override - public void reset() { - max = Long.MIN_VALUE; - } - -} http://git-wip-us.apache.org/repos/asf/accumulo/blob/b69291a3/core/src/main/java/org/apache/accumulo/core/iterators/aggregation/StringMin.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/org/apache/accumulo/core/iterators/aggregation/StringMin.java b/core/src/main/java/org/apache/accumulo/core/iterators/aggregation/StringMin.java deleted file mode 100644 index 7a49f81..0000000 --- a/core/src/main/java/org/apache/accumulo/core/iterators/aggregation/StringMin.java +++ /dev/null @@ -1,48 +0,0 @@ -/* - * 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.iterators.aggregation; - -import org.apache.accumulo.core.data.Value; - -/** - * @deprecated since 1.4, replaced by {@link org.apache.accumulo.core.iterators.user.MinCombiner} with - * {@link org.apache.accumulo.core.iterators.LongCombiner.Type#STRING} - */ -@Deprecated -public class StringMin implements Aggregator { - - long min = Long.MAX_VALUE; - - @Override - public Value aggregate() { - return new Value(Long.toString(min).getBytes()); - } - - @Override - public void collect(Value value) { - long l = Long.parseLong(new String(value.get())); - if (l < min) { - min = l; - } - } - - @Override - public void reset() { - min = Long.MAX_VALUE; - } - -} http://git-wip-us.apache.org/repos/asf/accumulo/blob/b69291a3/core/src/main/java/org/apache/accumulo/core/iterators/aggregation/StringSummation.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/org/apache/accumulo/core/iterators/aggregation/StringSummation.java b/core/src/main/java/org/apache/accumulo/core/iterators/aggregation/StringSummation.java deleted file mode 100644 index a8b5967..0000000 --- a/core/src/main/java/org/apache/accumulo/core/iterators/aggregation/StringSummation.java +++ /dev/null @@ -1,45 +0,0 @@ -/* - * 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.iterators.aggregation; - -import org.apache.accumulo.core.data.Value; - -/** - * @deprecated since 1.4, replaced by {@link org.apache.accumulo.core.iterators.user.SummingCombiner} with - * {@link org.apache.accumulo.core.iterators.LongCombiner.Type#STRING} - */ -@Deprecated -public class StringSummation implements Aggregator { - - long sum = 0; - - @Override - public Value aggregate() { - return new Value(Long.toString(sum).getBytes()); - } - - @Override - public void collect(Value value) { - sum += Long.parseLong(new String(value.get())); - } - - @Override - public void reset() { - sum = 0; - - } -} http://git-wip-us.apache.org/repos/asf/accumulo/blob/b69291a3/core/src/main/java/org/apache/accumulo/core/iterators/aggregation/conf/AggregatorConfiguration.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/org/apache/accumulo/core/iterators/aggregation/conf/AggregatorConfiguration.java b/core/src/main/java/org/apache/accumulo/core/iterators/aggregation/conf/AggregatorConfiguration.java deleted file mode 100644 index 3432cf5..0000000 --- a/core/src/main/java/org/apache/accumulo/core/iterators/aggregation/conf/AggregatorConfiguration.java +++ /dev/null @@ -1,34 +0,0 @@ -/* - * 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.iterators.aggregation.conf; - -import org.apache.hadoop.io.Text; - -/** - * @deprecated since 1.4 - */ -@Deprecated -public class AggregatorConfiguration extends org.apache.accumulo.core.iterators.conf.PerColumnIteratorConfig { - - public AggregatorConfiguration(Text columnFamily, String aggClassName) { - super(columnFamily, aggClassName); - } - - public AggregatorConfiguration(Text columnFamily, Text columnQualifier, String aggClassName) { - super(columnFamily, columnQualifier, aggClassName); - } -} http://git-wip-us.apache.org/repos/asf/accumulo/blob/b69291a3/core/src/main/java/org/apache/accumulo/core/iterators/aggregation/conf/AggregatorSet.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/org/apache/accumulo/core/iterators/aggregation/conf/AggregatorSet.java b/core/src/main/java/org/apache/accumulo/core/iterators/aggregation/conf/AggregatorSet.java deleted file mode 100644 index d6545ac..0000000 --- a/core/src/main/java/org/apache/accumulo/core/iterators/aggregation/conf/AggregatorSet.java +++ /dev/null @@ -1,41 +0,0 @@ -/* - * 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.iterators.aggregation.conf; - -import java.io.IOException; -import java.util.Map; - -import org.apache.accumulo.core.data.Key; -import org.apache.accumulo.core.iterators.conf.ColumnToClassMapping; - -/** - * @deprecated since 1.4 - */ -@Deprecated -public class AggregatorSet extends ColumnToClassMapping { - public AggregatorSet(Map opts) throws InstantiationException, IllegalAccessException, ClassNotFoundException, IOException { - super(opts, org.apache.accumulo.core.iterators.aggregation.Aggregator.class); - } - - public AggregatorSet() { - super(); - } - - public org.apache.accumulo.core.iterators.aggregation.Aggregator getAggregator(Key k) { - return getObject(k); - } -} http://git-wip-us.apache.org/repos/asf/accumulo/blob/b69291a3/core/src/main/java/org/apache/accumulo/core/iterators/conf/PerColumnIteratorConfig.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/org/apache/accumulo/core/iterators/conf/PerColumnIteratorConfig.java b/core/src/main/java/org/apache/accumulo/core/iterators/conf/PerColumnIteratorConfig.java deleted file mode 100644 index 310776aa..0000000 --- a/core/src/main/java/org/apache/accumulo/core/iterators/conf/PerColumnIteratorConfig.java +++ /dev/null @@ -1,81 +0,0 @@ -/* - * 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.iterators.conf; - -import org.apache.hadoop.io.Text; - -/** - * @deprecated since 1.4 - * - * @see org.apache.accumulo.core.client.IteratorSetting.Column - * @see org.apache.accumulo.core.iterators.Combiner#setColumns(org.apache.accumulo.core.client.IteratorSetting, java.util.List) - */ -@Deprecated -public class PerColumnIteratorConfig { - - private String parameter; - private Text colq; - private Text colf; - - public PerColumnIteratorConfig(Text columnFamily, String parameter) { - this.colf = columnFamily; - this.colq = null; - this.parameter = parameter; - } - - public PerColumnIteratorConfig(Text columnFamily, Text columnQualifier, String parameter) { - this.colf = columnFamily; - this.colq = columnQualifier; - this.parameter = parameter; - } - - public Text getColumnFamily() { - return colf; - } - - public Text getColumnQualifier() { - return colq; - } - - public String encodeColumns() { - return encodeColumns(this); - } - - public String getClassName() { - return parameter; - } - - private static String encodeColumns(PerColumnIteratorConfig pcic) { - return ColumnSet.encodeColumns(pcic.colf, pcic.colq); - } - - public static String encodeColumns(Text columnFamily, Text columnQualifier) { - return ColumnSet.encodeColumns(columnFamily, columnQualifier); - } - - public static PerColumnIteratorConfig decodeColumns(String columns, String className) { - String[] cols = columns.split(":"); - - if (cols.length == 1) { - return new PerColumnIteratorConfig(ColumnSet.decode(cols[0]), className); - } else if (cols.length == 2) { - return new PerColumnIteratorConfig(ColumnSet.decode(cols[0]), ColumnSet.decode(cols[1]), className); - } else { - throw new IllegalArgumentException(columns); - } - } -} http://git-wip-us.apache.org/repos/asf/accumulo/blob/b69291a3/core/src/main/java/org/apache/accumulo/core/iterators/user/IntersectingIterator.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/org/apache/accumulo/core/iterators/user/IntersectingIterator.java b/core/src/main/java/org/apache/accumulo/core/iterators/user/IntersectingIterator.java index 94995cb..f848b10 100644 --- a/core/src/main/java/org/apache/accumulo/core/iterators/user/IntersectingIterator.java +++ b/core/src/main/java/org/apache/accumulo/core/iterators/user/IntersectingIterator.java @@ -503,30 +503,6 @@ public class IntersectingIterator implements SortedKeyValueIterator { } /** - * @deprecated since 1.6.0 - */ - @Deprecated - public void addSource(SortedKeyValueIterator source, IteratorEnvironment env, Text term, boolean notFlag) { - // Check if we have space for the added Source - if (sources == null) { - sources = new TermSource[1]; - } else { - // allocate space for node, and copy current tree. - // TODO: Should we change this to an ArrayList so that we can just add() ? - ACCUMULO-1309 - TermSource[] localSources = new TermSource[sources.length + 1]; - int currSource = 0; - for (TermSource myTerm : sources) { - // TODO: Do I need to call new here? or can I just re-use the term? - ACCUMULO-1309 - localSources[currSource] = new TermSource(myTerm); - currSource++; - } - sources = localSources; - } - sources[sourcesCount] = new TermSource(source.deepCopy(env), term, notFlag); - sourcesCount++; - } - - /** * Encode the columns to be used when iterating. */ public static void setColumnFamilies(IteratorSetting cfg, Text[] columns) { http://git-wip-us.apache.org/repos/asf/accumulo/blob/b69291a3/core/src/main/java/org/apache/accumulo/core/replication/ReplicationTable.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/org/apache/accumulo/core/replication/ReplicationTable.java b/core/src/main/java/org/apache/accumulo/core/replication/ReplicationTable.java index 7076757..2479051 100644 --- a/core/src/main/java/org/apache/accumulo/core/replication/ReplicationTable.java +++ b/core/src/main/java/org/apache/accumulo/core/replication/ReplicationTable.java @@ -35,7 +35,6 @@ import org.apache.accumulo.core.master.state.tables.TableState; import org.apache.accumulo.core.replication.ReplicationSchema.StatusSection; import org.apache.accumulo.core.replication.ReplicationSchema.WorkSection; import org.apache.accumulo.core.security.Authorizations; -import org.apache.accumulo.core.util.DeprecationUtil; import org.apache.hadoop.io.Text; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -87,7 +86,7 @@ public class ReplicationTable { } public static boolean isOnline(Connector conn) { - return DeprecationUtil.isMockInstance(conn.getInstance()) || TableState.ONLINE == Tables.getTableState(conn.getInstance(), ID); + return TableState.ONLINE == Tables.getTableState(conn.getInstance(), ID); } public static void setOnline(Connector conn) throws AccumuloSecurityException, AccumuloException { http://git-wip-us.apache.org/repos/asf/accumulo/blob/b69291a3/core/src/main/java/org/apache/accumulo/core/security/VisibilityConstraint.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/org/apache/accumulo/core/security/VisibilityConstraint.java b/core/src/main/java/org/apache/accumulo/core/security/VisibilityConstraint.java deleted file mode 100644 index 611c8d4..0000000 --- a/core/src/main/java/org/apache/accumulo/core/security/VisibilityConstraint.java +++ /dev/null @@ -1,29 +0,0 @@ -/* - * 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.security; - -/** - * - * @deprecated since 1.7.0 This is server side code not intended to exist in a public API package. This class references types that are not in the public API - * and therefore is not guaranteed to be stable. It was deprecated to clearly communicate this. Use - * {@link org.apache.accumulo.core.constraints.VisibilityConstraint} instead. - */ -@Deprecated -public class VisibilityConstraint extends org.apache.accumulo.core.constraints.VisibilityConstraint { - -} http://git-wip-us.apache.org/repos/asf/accumulo/blob/b69291a3/core/src/main/java/org/apache/accumulo/core/security/crypto/CachingHDFSSecretKeyEncryptionStrategy.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/org/apache/accumulo/core/security/crypto/CachingHDFSSecretKeyEncryptionStrategy.java b/core/src/main/java/org/apache/accumulo/core/security/crypto/CachingHDFSSecretKeyEncryptionStrategy.java index 4ee27ef..b340abd 100644 --- a/core/src/main/java/org/apache/accumulo/core/security/crypto/CachingHDFSSecretKeyEncryptionStrategy.java +++ b/core/src/main/java/org/apache/accumulo/core/security/crypto/CachingHDFSSecretKeyEncryptionStrategy.java @@ -41,6 +41,8 @@ import org.slf4j.LoggerFactory; */ public class CachingHDFSSecretKeyEncryptionStrategy implements SecretKeyEncryptionStrategy { + @SuppressWarnings("deprecation") + private static final Property INSTANCE_DFS_DIR = Property.INSTANCE_DFS_DIR; private static final Logger log = LoggerFactory.getLogger(CachingHDFSSecretKeyEncryptionStrategy.class); private SecretKeyCache secretKeyCache = new SecretKeyCache(); @@ -173,17 +175,16 @@ public class CachingHDFSSecretKeyEncryptionStrategy implements SecretKeyEncrypti } - @SuppressWarnings("deprecation") private String getFullPathToKey(CryptoModuleParameters params) { String pathToKeyName = params.getAllOptions().get(Property.CRYPTO_DEFAULT_KEY_STRATEGY_KEY_LOCATION.getKey()); - String instanceDirectory = params.getAllOptions().get(Property.INSTANCE_DFS_DIR.getKey()); + String instanceDirectory = params.getAllOptions().get(INSTANCE_DFS_DIR.getKey()); if (pathToKeyName == null) { pathToKeyName = Property.CRYPTO_DEFAULT_KEY_STRATEGY_KEY_LOCATION.getDefaultValue(); } if (instanceDirectory == null) { - instanceDirectory = Property.INSTANCE_DFS_DIR.getDefaultValue(); + instanceDirectory = INSTANCE_DFS_DIR.getDefaultValue(); } if (!pathToKeyName.startsWith("/")) { http://git-wip-us.apache.org/repos/asf/accumulo/blob/b69291a3/core/src/main/java/org/apache/accumulo/core/security/crypto/NonCachingSecretKeyEncryptionStrategy.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/org/apache/accumulo/core/security/crypto/NonCachingSecretKeyEncryptionStrategy.java b/core/src/main/java/org/apache/accumulo/core/security/crypto/NonCachingSecretKeyEncryptionStrategy.java index 1dd8d60..f0eaa26 100644 --- a/core/src/main/java/org/apache/accumulo/core/security/crypto/NonCachingSecretKeyEncryptionStrategy.java +++ b/core/src/main/java/org/apache/accumulo/core/security/crypto/NonCachingSecretKeyEncryptionStrategy.java @@ -39,6 +39,10 @@ import org.slf4j.LoggerFactory; //TODO ACCUMULO-2530 Update properties to use a URI instead of a relative path to secret key public class NonCachingSecretKeyEncryptionStrategy implements SecretKeyEncryptionStrategy { + @SuppressWarnings("deprecation") + private static final Property INSTANCE_DFS_DIR = Property.INSTANCE_DFS_DIR; + @SuppressWarnings("deprecation") + private static final Property INSTANCE_DFS_URI = Property.INSTANCE_DFS_URI; private static final Logger log = LoggerFactory.getLogger(NonCachingSecretKeyEncryptionStrategy.class); private void doKeyEncryptionOperation(int encryptionMode, CryptoModuleParameters params, String pathToKeyName, Path pathToKey, FileSystem fs) @@ -121,17 +125,16 @@ public class NonCachingSecretKeyEncryptionStrategy implements SecretKeyEncryptio } } - @SuppressWarnings("deprecation") private String getFullPathToKey(CryptoModuleParameters params) { String pathToKeyName = params.getAllOptions().get(Property.CRYPTO_DEFAULT_KEY_STRATEGY_KEY_LOCATION.getKey()); - String instanceDirectory = params.getAllOptions().get(Property.INSTANCE_DFS_DIR.getKey()); + String instanceDirectory = params.getAllOptions().get(INSTANCE_DFS_DIR.getKey()); if (pathToKeyName == null) { pathToKeyName = Property.CRYPTO_DEFAULT_KEY_STRATEGY_KEY_LOCATION.getDefaultValue(); } if (instanceDirectory == null) { - instanceDirectory = Property.INSTANCE_DFS_DIR.getDefaultValue(); + instanceDirectory = INSTANCE_DFS_DIR.getDefaultValue(); } if (!pathToKeyName.startsWith("/")) { @@ -142,12 +145,11 @@ public class NonCachingSecretKeyEncryptionStrategy implements SecretKeyEncryptio return fullPath; } - @SuppressWarnings("deprecation") @Override public CryptoModuleParameters encryptSecretKey(CryptoModuleParameters params) { - String hdfsURI = params.getAllOptions().get(Property.INSTANCE_DFS_URI.getKey()); + String hdfsURI = params.getAllOptions().get(INSTANCE_DFS_URI.getKey()); if (hdfsURI == null) { - hdfsURI = Property.INSTANCE_DFS_URI.getDefaultValue(); + hdfsURI = INSTANCE_DFS_URI.getDefaultValue(); } String fullPath = getFullPathToKey(params); @@ -166,12 +168,11 @@ public class NonCachingSecretKeyEncryptionStrategy implements SecretKeyEncryptio return params; } - @SuppressWarnings("deprecation") @Override public CryptoModuleParameters decryptSecretKey(CryptoModuleParameters params) { - String hdfsURI = params.getAllOptions().get(Property.INSTANCE_DFS_URI.getKey()); + String hdfsURI = params.getAllOptions().get(INSTANCE_DFS_URI.getKey()); if (hdfsURI == null) { - hdfsURI = Property.INSTANCE_DFS_URI.getDefaultValue(); + hdfsURI = INSTANCE_DFS_URI.getDefaultValue(); } String pathToKeyName = getFullPathToKey(params); http://git-wip-us.apache.org/repos/asf/accumulo/blob/b69291a3/core/src/main/java/org/apache/accumulo/core/trace/DistributedTrace.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/org/apache/accumulo/core/trace/DistributedTrace.java b/core/src/main/java/org/apache/accumulo/core/trace/DistributedTrace.java index 14886f1..f2a6941 100644 --- a/core/src/main/java/org/apache/accumulo/core/trace/DistributedTrace.java +++ b/core/src/main/java/org/apache/accumulo/core/trace/DistributedTrace.java @@ -23,16 +23,13 @@ import java.util.Map.Entry; import org.apache.accumulo.core.client.ClientConfiguration; import org.apache.accumulo.core.client.ClientConfiguration.ClientProperty; -import org.apache.accumulo.core.client.Instance; import org.apache.accumulo.core.conf.AccumuloConfiguration; import org.apache.accumulo.core.conf.Property; -import org.apache.accumulo.fate.zookeeper.ZooReader; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.util.ShutdownHookManager; import org.apache.htrace.HTraceConfiguration; import org.apache.htrace.SpanReceiver; import org.apache.htrace.SpanReceiverBuilder; -import org.apache.zookeeper.KeeperException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -55,14 +52,6 @@ public class DistributedTrace { private static final HashSet receivers = new HashSet(); /** - * @deprecated since 1.7, use {@link DistributedTrace#enable(String, String, org.apache.accumulo.core.client.ClientConfiguration)} instead - */ - @Deprecated - public static void enable(Instance instance, ZooReader zoo, String application, String address) throws IOException, KeeperException, InterruptedException { - enable(address, application); - } - - /** * Enable tracing by setting up SpanReceivers for the current process. */ public static void enable() { http://git-wip-us.apache.org/repos/asf/accumulo/blob/b69291a3/core/src/main/java/org/apache/accumulo/core/trace/Trace.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/org/apache/accumulo/core/trace/Trace.java b/core/src/main/java/org/apache/accumulo/core/trace/Trace.java index 3ebd031..35227c5 100644 --- a/core/src/main/java/org/apache/accumulo/core/trace/Trace.java +++ b/core/src/main/java/org/apache/accumulo/core/trace/Trace.java @@ -56,14 +56,6 @@ public class Trace { } /** - * @deprecated since 1.7, use {@link #off()} instead - */ - @Deprecated - public static void offNoFlush() { - off(); - } - - /** * Returns whether tracing is currently on. */ public static boolean isTracing() { @@ -71,16 +63,6 @@ public class Trace { } /** - * Return the current span. - * - * @deprecated since 1.7 -- it is better to save the span you create in a local variable and call its methods, rather than retrieving the current span - */ - @Deprecated - public static Span currentTrace() { - return new Span(org.apache.htrace.Trace.currentSpan()); - } - - /** * Get the trace id of the current span. */ public static long currentTraceId() { http://git-wip-us.apache.org/repos/asf/accumulo/blob/b69291a3/core/src/main/java/org/apache/accumulo/core/util/DeprecationUtil.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/org/apache/accumulo/core/util/DeprecationUtil.java b/core/src/main/java/org/apache/accumulo/core/util/DeprecationUtil.java deleted file mode 100644 index cd798bb..0000000 --- a/core/src/main/java/org/apache/accumulo/core/util/DeprecationUtil.java +++ /dev/null @@ -1,61 +0,0 @@ -/* - * 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.util; - -import org.apache.accumulo.core.client.Instance; -import org.apache.accumulo.core.client.impl.TabletLocator; -import org.apache.accumulo.core.client.mapreduce.RangeInputSplit; - -/** - * A utility class for managing deprecated items. This avoids scattering private helper methods all over the code with warnings suppression. - * - *

- * This class will never be public API and methods will be removed as soon as they are no longer needed. No methods in this class will, themselves, be - * deprecated, because that would propagate the deprecation warning we are trying to avoid. - * - *

- * This class should not be used as a substitute for deprecated classes. It should only be used for implementation code which must remain to support the - * deprecated features, and only until that feature is removed. - */ -public class DeprecationUtil { - - @SuppressWarnings("deprecation") - public static boolean isMockInstance(Instance instance) { - return instance instanceof org.apache.accumulo.core.client.mock.MockInstance; - } - - @SuppressWarnings("deprecation") - public static Instance makeMockInstance(String instance) { - return new org.apache.accumulo.core.client.mock.MockInstance(instance); - } - - @SuppressWarnings("deprecation") - public static void setMockInstance(RangeInputSplit split, boolean isMockInstance) { - split.setMockInstance(isMockInstance); - } - - @SuppressWarnings("deprecation") - public static boolean isMockInstanceSet(RangeInputSplit split) { - return split.isMockInstance(); - } - - @SuppressWarnings("deprecation") - public static TabletLocator makeMockLocator() { - return new org.apache.accumulo.core.client.mock.impl.MockTabletLocator(); - } - -} http://git-wip-us.apache.org/repos/asf/accumulo/blob/b69291a3/core/src/main/java/org/apache/accumulo/core/util/format/BinaryFormatter.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/org/apache/accumulo/core/util/format/BinaryFormatter.java b/core/src/main/java/org/apache/accumulo/core/util/format/BinaryFormatter.java deleted file mode 100644 index f5cbe39..0000000 --- a/core/src/main/java/org/apache/accumulo/core/util/format/BinaryFormatter.java +++ /dev/null @@ -1,66 +0,0 @@ -/* - * 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.util.format; - -import java.util.Map.Entry; -import org.apache.accumulo.core.data.Key; -import org.apache.accumulo.core.data.Value; -import org.apache.accumulo.core.security.ColumnVisibility; - -/** - * @deprecated Use {@link DefaultFormatter} providing showLength and printTimestamps via {@link FormatterConfig}. - */ -@Deprecated -public class BinaryFormatter extends DefaultFormatter { - // this class can probably be replaced by DefaultFormatter since DefaultFormatter has the max length stuff - @Override - public String next() { - checkState(true); - return formatEntry(getScannerIterator().next(), config.willPrintTimestamps(), config.getShownLength()); - } - - public static String formatEntry(Entry entry, boolean printTimestamps, int shownLength) { - StringBuilder sb = new StringBuilder(); - - Key key = entry.getKey(); - - // append row - appendText(sb, key.getRow(), shownLength).append(" "); - - // append column family - appendText(sb, key.getColumnFamily(), shownLength).append(":"); - - // append column qualifier - appendText(sb, key.getColumnQualifier(), shownLength).append(" "); - - // append visibility expression - sb.append(new ColumnVisibility(key.getColumnVisibility())); - - // append timestamp - if (printTimestamps) - sb.append(" ").append(entry.getKey().getTimestamp()); - - // append value - Value value = entry.getValue(); - if (value != null && value.getSize() > 0) { - sb.append("\t"); - appendValue(sb, value, shownLength); - } - return sb.toString(); - } - -} http://git-wip-us.apache.org/repos/asf/accumulo/blob/b69291a3/core/src/main/java/org/apache/accumulo/core/util/format/DateStringFormatter.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/org/apache/accumulo/core/util/format/DateStringFormatter.java b/core/src/main/java/org/apache/accumulo/core/util/format/DateStringFormatter.java deleted file mode 100644 index 63bd536..0000000 --- a/core/src/main/java/org/apache/accumulo/core/util/format/DateStringFormatter.java +++ /dev/null @@ -1,75 +0,0 @@ -/* - * 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.util.format; - -import java.util.Map.Entry; -import java.util.TimeZone; -import org.apache.accumulo.core.data.Key; -import org.apache.accumulo.core.data.Value; - -/** - * This class is not recommended because {@link #initialize(Iterable, FormatterConfig)} replaces parameters in {@link FormatterConfig}, which - * could surprise users. - * - * This class can be replaced by {@link DefaultFormatter} where FormatterConfig is initialized with a DateFormat set to {@link #DATE_FORMAT}. See - * {@link DateFormatSupplier#createSimpleFormatSupplier(String, java.util.TimeZone)}. - * - *

- * final DateFormatSupplier dfSupplier = DateFormatSupplier.createSimpleFormatSupplier(DateFormatSupplier.HUMAN_READABLE_FORMAT, TimeZone.getTimeZone("UTC"));
- * final FormatterConfig config = new FormatterConfig().setPrintTimestamps(true).setDateFormatSupplier(dfSupplier);
- * 
- */ -@Deprecated -public class DateStringFormatter implements Formatter { - - private DefaultFormatter defaultFormatter; - private TimeZone timeZone; - - public static final String DATE_FORMAT = DateFormatSupplier.HUMAN_READABLE_FORMAT; - - public DateStringFormatter() { - this(TimeZone.getDefault()); - } - - public DateStringFormatter(TimeZone timeZone) { - this.defaultFormatter = new DefaultFormatter(); - this.timeZone = timeZone; - } - - @Override - public void initialize(Iterable> scanner, FormatterConfig config) { - FormatterConfig newConfig = new FormatterConfig(config); - newConfig.setDateFormatSupplier(DateFormatSupplier.createSimpleFormatSupplier(DATE_FORMAT, timeZone)); - defaultFormatter.initialize(scanner, newConfig); - } - - @Override - public boolean hasNext() { - return defaultFormatter.hasNext(); - } - - @Override - public String next() { - return defaultFormatter.next(); - } - - @Override - public void remove() { - defaultFormatter.remove(); - } - -} http://git-wip-us.apache.org/repos/asf/accumulo/blob/b69291a3/core/src/main/java/org/apache/accumulo/core/volume/VolumeConfiguration.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/org/apache/accumulo/core/volume/VolumeConfiguration.java b/core/src/main/java/org/apache/accumulo/core/volume/VolumeConfiguration.java index 573978d..31cf53c 100644 --- a/core/src/main/java/org/apache/accumulo/core/volume/VolumeConfiguration.java +++ b/core/src/main/java/org/apache/accumulo/core/volume/VolumeConfiguration.java @@ -31,6 +31,11 @@ import org.apache.hadoop.fs.Path; public class VolumeConfiguration { + @SuppressWarnings("deprecation") + private static final Property INSTANCE_DFS_DIR = Property.INSTANCE_DFS_DIR; + @SuppressWarnings("deprecation") + private static final Property INSTANCE_DFS_URI = Property.INSTANCE_DFS_URI; + public static Volume getVolume(String path, Configuration conf, AccumuloConfiguration acuconf) throws IOException { requireNonNull(path); @@ -44,8 +49,7 @@ public class VolumeConfiguration { } public static Volume getDefaultVolume(Configuration conf, AccumuloConfiguration acuconf) throws IOException { - @SuppressWarnings("deprecation") - String uri = acuconf.get(Property.INSTANCE_DFS_URI); + String uri = acuconf.get(INSTANCE_DFS_URI); // By default pull from INSTANCE_DFS_URI, falling back to the Hadoop defined // default filesystem (fs.defaultFS or the deprecated fs.default.name) @@ -60,12 +64,14 @@ public class VolumeConfiguration { } /** - * @see org.apache.accumulo.core.volume.VolumeConfiguration#getVolumeUris(AccumuloConfiguration,Configuration) + * This method gets the old configured base directory, using the URI and DIR. It will not longer be needed when we no longer support upgrading from non-volume + * based Accumulo config + * + * @see #getVolumeUris(AccumuloConfiguration,Configuration) */ - @Deprecated - public static String getConfiguredBaseDir(AccumuloConfiguration conf, Configuration hadoopConfig) { - String singleNamespace = conf.get(Property.INSTANCE_DFS_DIR); - String dfsUri = conf.get(Property.INSTANCE_DFS_URI); + private static String getConfiguredBaseDir(AccumuloConfiguration conf, Configuration hadoopConfig) { + String singleNamespace = conf.get(INSTANCE_DFS_DIR); + String dfsUri = conf.get(INSTANCE_DFS_URI); String baseDir; if (dfsUri == null || dfsUri.isEmpty()) { @@ -76,7 +82,7 @@ public class VolumeConfiguration { } } else { if (!dfsUri.contains(":")) - throw new IllegalArgumentException("Expected fully qualified URI for " + Property.INSTANCE_DFS_URI.getKey() + " got " + dfsUri); + throw new IllegalArgumentException("Expected fully qualified URI for " + INSTANCE_DFS_URI.getKey() + " got " + dfsUri); baseDir = dfsUri + singleNamespace; } return baseDir; @@ -140,10 +146,9 @@ public class VolumeConfiguration { * A FileSystem to write to * @return A Volume instance writing to the given FileSystem in the default path */ - @SuppressWarnings("deprecation") public static Volume create(T fs, AccumuloConfiguration acuconf) { - String dfsDir = acuconf.get(Property.INSTANCE_DFS_DIR); - return new VolumeImpl(fs, null == dfsDir ? Property.INSTANCE_DFS_DIR.getDefaultValue() : dfsDir); + String dfsDir = acuconf.get(INSTANCE_DFS_DIR); + return new VolumeImpl(fs, null == dfsDir ? INSTANCE_DFS_DIR.getDefaultValue() : dfsDir); } public static Volume create(T fs, String basePath) { http://git-wip-us.apache.org/repos/asf/accumulo/blob/b69291a3/core/src/test/java/org/apache/accumulo/core/cli/TestClientOpts.java ---------------------------------------------------------------------- diff --git a/core/src/test/java/org/apache/accumulo/core/cli/TestClientOpts.java b/core/src/test/java/org/apache/accumulo/core/cli/TestClientOpts.java index 65df5c9..638f152 100644 --- a/core/src/test/java/org/apache/accumulo/core/cli/TestClientOpts.java +++ b/core/src/test/java/org/apache/accumulo/core/cli/TestClientOpts.java @@ -50,6 +50,10 @@ import org.junit.rules.TestName; import com.beust.jcommander.JCommander; public class TestClientOpts { + @SuppressWarnings("deprecation") + private static final Property INSTANCE_DFS_DIR = Property.INSTANCE_DFS_DIR; + @SuppressWarnings("deprecation") + private static final Property INSTANCE_DFS_URI = Property.INSTANCE_DFS_URI; @Rule public TemporaryFolder tmpDir = new TemporaryFolder(new File(System.getProperty("user.dir") + "/target")); @@ -133,7 +137,6 @@ public class TestClientOpts { args.getInstance(); } - @SuppressWarnings("deprecation") @Test public void testInstanceDir() throws IOException { File instanceId = tmpDir.newFolder("instance_id"); @@ -146,9 +149,8 @@ public class TestClientOpts { FileWriter fileWriter = new FileWriter(siteXml); fileWriter.append("\n"); - fileWriter - .append("" + Property.INSTANCE_DFS_DIR.getKey() + "" + tmpDir.getRoot().getAbsolutePath() + "\n"); - fileWriter.append("" + Property.INSTANCE_DFS_URI.getKey() + "file://\n"); + fileWriter.append("" + INSTANCE_DFS_DIR.getKey() + "" + tmpDir.getRoot().getAbsolutePath() + "\n"); + fileWriter.append("" + INSTANCE_DFS_URI.getKey() + "file://\n"); fileWriter.append("" + ClientProperty.INSTANCE_NAME + "foo\n"); fileWriter.append("\n"); http://git-wip-us.apache.org/repos/asf/accumulo/blob/b69291a3/core/src/test/java/org/apache/accumulo/core/client/impl/TableOperationsHelperTest.java ---------------------------------------------------------------------- diff --git a/core/src/test/java/org/apache/accumulo/core/client/impl/TableOperationsHelperTest.java b/core/src/test/java/org/apache/accumulo/core/client/impl/TableOperationsHelperTest.java index c55b62f..d6615a4 100644 --- a/core/src/test/java/org/apache/accumulo/core/client/impl/TableOperationsHelperTest.java +++ b/core/src/test/java/org/apache/accumulo/core/client/impl/TableOperationsHelperTest.java @@ -37,7 +37,6 @@ import org.apache.accumulo.core.client.admin.CompactionConfig; import org.apache.accumulo.core.client.admin.DiskUsage; import org.apache.accumulo.core.client.admin.Locations; import org.apache.accumulo.core.client.admin.NewTableConfiguration; -import org.apache.accumulo.core.client.admin.TimeType; import org.apache.accumulo.core.client.sample.SamplerConfiguration; import org.apache.accumulo.core.data.Range; import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope; @@ -65,33 +64,11 @@ public class TableOperationsHelperTest { public void create(String tableName) throws AccumuloException, AccumuloSecurityException, TableExistsException {} @Override - @Deprecated - public void create(String tableName, boolean limitVersion) throws AccumuloException, AccumuloSecurityException, TableExistsException { - create(tableName, limitVersion, TimeType.MILLIS); - } - - @Override - @Deprecated - public void create(String tableName, boolean versioningIter, TimeType timeType) throws AccumuloException, AccumuloSecurityException, TableExistsException {} - - @Override public void create(String tableName, NewTableConfiguration ntc) throws AccumuloException, AccumuloSecurityException, TableExistsException {} @Override public void addSplits(String tableName, SortedSet partitionKeys) throws TableNotFoundException, AccumuloException, AccumuloSecurityException {} - @Deprecated - @Override - public Collection getSplits(String tableName) throws TableNotFoundException { - return null; - } - - @Deprecated - @Override - public Collection getSplits(String tableName, int maxSplits) throws TableNotFoundException { - return null; - } - @Override public Collection listSplits(String tableName) throws TableNotFoundException { return null; @@ -138,10 +115,6 @@ public class TableOperationsHelperTest { public void rename(String oldTableName, String newTableName) throws AccumuloSecurityException, TableNotFoundException, AccumuloException, TableExistsException {} - @Deprecated - @Override - public void flush(String tableName) throws AccumuloException, AccumuloSecurityException {} - @Override public void flush(String tableName, Text start, Text end, boolean wait) throws AccumuloException, AccumuloSecurityException, TableNotFoundException {} http://git-wip-us.apache.org/repos/asf/accumulo/blob/b69291a3/core/src/test/java/org/apache/accumulo/core/client/impl/TabletLocatorImplTest.java ---------------------------------------------------------------------- diff --git a/core/src/test/java/org/apache/accumulo/core/client/impl/TabletLocatorImplTest.java b/core/src/test/java/org/apache/accumulo/core/client/impl/TabletLocatorImplTest.java index a053cef..fbeb69b 100644 --- a/core/src/test/java/org/apache/accumulo/core/client/impl/TabletLocatorImplTest.java +++ b/core/src/test/java/org/apache/accumulo/core/client/impl/TabletLocatorImplTest.java @@ -21,7 +21,6 @@ import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; -import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; @@ -45,7 +44,6 @@ import org.apache.accumulo.core.client.impl.TabletLocator.TabletServerMutations; import org.apache.accumulo.core.client.impl.TabletLocatorImpl.TabletLocationObtainer; import org.apache.accumulo.core.client.impl.TabletLocatorImpl.TabletServerLockChecker; import org.apache.accumulo.core.client.security.tokens.AuthenticationToken; -import org.apache.accumulo.core.conf.AccumuloConfiguration; import org.apache.accumulo.core.data.Key; import org.apache.accumulo.core.data.Mutation; import org.apache.accumulo.core.data.PartialKey; @@ -85,7 +83,6 @@ public class TabletLocatorImplTest { return objs; } - @SuppressWarnings("unchecked") static Map>> createExpectedBinnings(Object... data) { Map>> expBinnedRanges = new HashMap>>(); @@ -100,6 +97,7 @@ public class TabletLocatorImplTest { for (int j = 0; j < binData.length; j += 2) { KeyExtent ke = (KeyExtent) binData[j]; + @SuppressWarnings("unchecked") List ranges = (List) binData[j + 1]; binnedKE.put(ke, ranges); @@ -440,36 +438,6 @@ public class TabletLocatorImplTest { } @Override - @Deprecated - public Connector getConnector(String user, byte[] pass) throws AccumuloException, AccumuloSecurityException { - throw new UnsupportedOperationException(); - } - - @Override - @Deprecated - public Connector getConnector(String user, ByteBuffer pass) throws AccumuloException, AccumuloSecurityException { - throw new UnsupportedOperationException(); - } - - @Deprecated - @Override - public AccumuloConfiguration getConfiguration() { - throw new UnsupportedOperationException(); - } - - @Override - @Deprecated - public void setConfiguration(AccumuloConfiguration conf) { - throw new UnsupportedOperationException(); - } - - @Override - @Deprecated - public Connector getConnector(String user, CharSequence pass) throws AccumuloException, AccumuloSecurityException { - throw new UnsupportedOperationException(); - } - - @Override public Connector getConnector(String principal, AuthenticationToken token) throws AccumuloException, AccumuloSecurityException { throw new UnsupportedOperationException(); } http://git-wip-us.apache.org/repos/asf/accumulo/blob/b69291a3/core/src/test/java/org/apache/accumulo/core/client/mapred/RangeInputSplitTest.java ---------------------------------------------------------------------- diff --git a/core/src/test/java/org/apache/accumulo/core/client/mapred/RangeInputSplitTest.java b/core/src/test/java/org/apache/accumulo/core/client/mapred/RangeInputSplitTest.java index 6c75ec2..458cbf5 100644 --- a/core/src/test/java/org/apache/accumulo/core/client/mapred/RangeInputSplitTest.java +++ b/core/src/test/java/org/apache/accumulo/core/client/mapred/RangeInputSplitTest.java @@ -33,7 +33,6 @@ import org.apache.accumulo.core.data.Range; import org.apache.accumulo.core.iterators.user.SummingCombiner; import org.apache.accumulo.core.iterators.user.WholeRowIterator; import org.apache.accumulo.core.security.Authorizations; -import org.apache.accumulo.core.util.DeprecationUtil; import org.apache.accumulo.core.util.Pair; import org.apache.hadoop.io.Text; import org.apache.log4j.Level; @@ -87,7 +86,6 @@ public class RangeInputSplitTest { split.setToken(new PasswordToken("password")); split.setPrincipal("root"); split.setInstanceName("instance"); - DeprecationUtil.setMockInstance(split, true); split.setZooKeepers("localhost"); split.setIterators(iterators); split.setLogLevel(Level.WARN); @@ -113,7 +111,6 @@ public class RangeInputSplitTest { Assert.assertEquals(split.getToken(), newSplit.getToken()); Assert.assertEquals(split.getPrincipal(), newSplit.getPrincipal()); Assert.assertEquals(split.getInstanceName(), newSplit.getInstanceName()); - Assert.assertEquals(DeprecationUtil.isMockInstanceSet(split), DeprecationUtil.isMockInstanceSet(newSplit)); Assert.assertEquals(split.getZooKeepers(), newSplit.getZooKeepers()); Assert.assertEquals(split.getIterators(), newSplit.getIterators()); Assert.assertEquals(split.getLogLevel(), newSplit.getLogLevel()); http://git-wip-us.apache.org/repos/asf/accumulo/blob/b69291a3/core/src/test/java/org/apache/accumulo/core/client/mapreduce/RangeInputSplitTest.java ---------------------------------------------------------------------- diff --git a/core/src/test/java/org/apache/accumulo/core/client/mapreduce/RangeInputSplitTest.java b/core/src/test/java/org/apache/accumulo/core/client/mapreduce/RangeInputSplitTest.java index 1cf8c55..ce03022 100644 --- a/core/src/test/java/org/apache/accumulo/core/client/mapreduce/RangeInputSplitTest.java +++ b/core/src/test/java/org/apache/accumulo/core/client/mapreduce/RangeInputSplitTest.java @@ -33,7 +33,6 @@ import org.apache.accumulo.core.data.Range; import org.apache.accumulo.core.iterators.user.SummingCombiner; import org.apache.accumulo.core.iterators.user.WholeRowIterator; import org.apache.accumulo.core.security.Authorizations; -import org.apache.accumulo.core.util.DeprecationUtil; import org.apache.accumulo.core.util.Pair; import org.apache.hadoop.io.Text; import org.apache.log4j.Level; @@ -90,7 +89,6 @@ public class RangeInputSplitTest { split.setToken(new PasswordToken("password")); split.setPrincipal("root"); split.setInstanceName("instance"); - DeprecationUtil.setMockInstance(split, true); split.setZooKeepers("localhost"); split.setIterators(iterators); split.setLogLevel(Level.WARN); @@ -117,7 +115,6 @@ public class RangeInputSplitTest { Assert.assertEquals(split.getToken(), newSplit.getToken()); Assert.assertEquals(split.getPrincipal(), newSplit.getPrincipal()); Assert.assertEquals(split.getInstanceName(), newSplit.getInstanceName()); - Assert.assertEquals(DeprecationUtil.isMockInstanceSet(split), DeprecationUtil.isMockInstanceSet(newSplit)); Assert.assertEquals(split.getZooKeepers(), newSplit.getZooKeepers()); Assert.assertEquals(split.getIterators(), newSplit.getIterators()); Assert.assertEquals(split.getLogLevel(), newSplit.getLogLevel()); http://git-wip-us.apache.org/repos/asf/accumulo/blob/b69291a3/core/src/test/java/org/apache/accumulo/core/client/mapreduce/impl/BatchInputSplitTest.java ---------------------------------------------------------------------- diff --git a/core/src/test/java/org/apache/accumulo/core/client/mapreduce/impl/BatchInputSplitTest.java b/core/src/test/java/org/apache/accumulo/core/client/mapreduce/impl/BatchInputSplitTest.java index 74c3438..5ac2d30 100644 --- a/core/src/test/java/org/apache/accumulo/core/client/mapreduce/impl/BatchInputSplitTest.java +++ b/core/src/test/java/org/apache/accumulo/core/client/mapreduce/impl/BatchInputSplitTest.java @@ -33,7 +33,6 @@ import org.apache.accumulo.core.data.Range; import org.apache.accumulo.core.iterators.user.SummingCombiner; import org.apache.accumulo.core.iterators.user.WholeRowIterator; import org.apache.accumulo.core.security.Authorizations; -import org.apache.accumulo.core.util.DeprecationUtil; import org.apache.accumulo.core.util.Pair; import org.apache.hadoop.io.Text; import org.apache.log4j.Level; @@ -88,7 +87,6 @@ public class BatchInputSplitTest { split.setFetchedColumns(fetchedColumns); split.setToken(new PasswordToken("password")); split.setPrincipal("root"); - DeprecationUtil.setMockInstance(split, true); split.setInstanceName("instance"); split.setZooKeepers("localhost"); split.setIterators(iterators); @@ -113,7 +111,6 @@ public class BatchInputSplitTest { Assert.assertEquals(split.getToken(), newSplit.getToken()); Assert.assertEquals(split.getPrincipal(), newSplit.getPrincipal()); Assert.assertEquals(split.getInstanceName(), newSplit.getInstanceName()); - Assert.assertEquals(DeprecationUtil.isMockInstanceSet(split), DeprecationUtil.isMockInstanceSet(newSplit)); Assert.assertEquals(split.getZooKeepers(), newSplit.getZooKeepers()); Assert.assertEquals(split.getIterators(), newSplit.getIterators()); Assert.assertEquals(split.getLogLevel(), newSplit.getLogLevel()); http://git-wip-us.apache.org/repos/asf/accumulo/blob/b69291a3/core/src/test/java/org/apache/accumulo/core/client/mapreduce/lib/impl/ConfiguratorBaseTest.java ---------------------------------------------------------------------- diff --git a/core/src/test/java/org/apache/accumulo/core/client/mapreduce/lib/impl/ConfiguratorBaseTest.java b/core/src/test/java/org/apache/accumulo/core/client/mapreduce/lib/impl/ConfiguratorBaseTest.java index 7b5ac26..1eb29e3 100644 --- a/core/src/test/java/org/apache/accumulo/core/client/mapreduce/lib/impl/ConfiguratorBaseTest.java +++ b/core/src/test/java/org/apache/accumulo/core/client/mapreduce/lib/impl/ConfiguratorBaseTest.java @@ -26,7 +26,6 @@ import org.apache.accumulo.core.Constants; import org.apache.accumulo.core.client.AccumuloSecurityException; import org.apache.accumulo.core.client.ClientConfiguration; import org.apache.accumulo.core.client.ClientConfiguration.ClientProperty; -import org.apache.accumulo.core.client.Instance; import org.apache.accumulo.core.client.ZooKeeperInstance; import org.apache.accumulo.core.client.security.tokens.AuthenticationToken; import org.apache.accumulo.core.client.security.tokens.AuthenticationToken.AuthenticationTokenSerializer; @@ -101,19 +100,6 @@ public class ConfiguratorBaseTest { // assertEquals(1234000, ((ZooKeeperInstance) instance).getZooKeepersSessionTimeOut()); } - @SuppressWarnings("deprecation") - @Test - public void testSetMockInstance() { - Class mockClass = org.apache.accumulo.core.client.mock.MockInstance.class; - Configuration conf = new Configuration(); - ConfiguratorBase.setMockInstance(this.getClass(), conf, "testInstanceName"); - assertEquals("testInstanceName", conf.get(ConfiguratorBase.enumToConfKey(this.getClass(), ConfiguratorBase.InstanceOpts.NAME))); - assertEquals(null, conf.get(ConfiguratorBase.enumToConfKey(this.getClass(), ConfiguratorBase.InstanceOpts.ZOO_KEEPERS))); - assertEquals(mockClass.getSimpleName(), conf.get(ConfiguratorBase.enumToConfKey(this.getClass(), ConfiguratorBase.InstanceOpts.TYPE))); - Instance instance = ConfiguratorBase.getInstance(this.getClass(), conf); - assertEquals(mockClass.getName(), instance.getClass().getName()); - } - @Test public void testSetLogLevel() { Configuration conf = new Configuration(); http://git-wip-us.apache.org/repos/asf/accumulo/blob/b69291a3/core/src/test/java/org/apache/accumulo/core/client/mock/MockConnectorTest.java ---------------------------------------------------------------------- diff --git a/core/src/test/java/org/apache/accumulo/core/client/mock/MockConnectorTest.java b/core/src/test/java/org/apache/accumulo/core/client/mock/MockConnectorTest.java deleted file mode 100644 index 4a78a4b..0000000 --- a/core/src/test/java/org/apache/accumulo/core/client/mock/MockConnectorTest.java +++ /dev/null @@ -1,375 +0,0 @@ -/* - * 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.mock; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; - -import java.util.ArrayList; -import java.util.Collections; -import java.util.Iterator; -import java.util.List; -import java.util.Map.Entry; -import java.util.Random; -import java.util.concurrent.TimeUnit; - -import org.apache.accumulo.core.client.AccumuloException; -import org.apache.accumulo.core.client.AccumuloSecurityException; -import org.apache.accumulo.core.client.BatchDeleter; -import org.apache.accumulo.core.client.BatchScanner; -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.IteratorSetting; -import org.apache.accumulo.core.client.MultiTableBatchWriter; -import org.apache.accumulo.core.client.Scanner; -import org.apache.accumulo.core.client.security.tokens.PasswordToken; -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.iterators.Combiner; -import org.apache.accumulo.core.iterators.user.SummingCombiner; -import org.apache.accumulo.core.security.Authorizations; -import org.apache.hadoop.io.Text; -import org.junit.Assert; -import org.junit.Test; - -import com.google.common.collect.Iterators; - -@Deprecated -public class MockConnectorTest { - Random random = new Random(); - - static Text asText(int i) { - return new Text(Integer.toHexString(i)); - } - - @Test - public void testSunnyDay() throws Exception { - Connector c = new MockConnector("root", new MockInstance()); - c.tableOperations().create("test"); - BatchWriter bw = c.createBatchWriter("test", new BatchWriterConfig()); - for (int i = 0; i < 100; i++) { - int r = random.nextInt(); - Mutation m = new Mutation(asText(r)); - m.put(asText(random.nextInt()), asText(random.nextInt()), new Value(Integer.toHexString(r).getBytes())); - bw.addMutation(m); - } - bw.close(); - BatchScanner s = c.createBatchScanner("test", Authorizations.EMPTY, 2); - s.setRanges(Collections.singletonList(new Range())); - Key key = null; - int count = 0; - for (Entry entry : s) { - if (key != null) - assertTrue(key.compareTo(entry.getKey()) < 0); - assertEquals(entry.getKey().getRow(), new Text(entry.getValue().get())); - key = entry.getKey(); - count++; - } - assertEquals(100, count); - } - - @Test - public void testChangeAuths() throws Exception { - Connector c = new MockConnector("root", new MockInstance()); - c.securityOperations().createLocalUser("greg", new PasswordToken(new byte[0])); - assertTrue(c.securityOperations().getUserAuthorizations("greg").isEmpty()); - c.securityOperations().changeUserAuthorizations("greg", new Authorizations("A".getBytes())); - assertTrue(c.securityOperations().getUserAuthorizations("greg").contains("A".getBytes())); - c.securityOperations().changeUserAuthorizations("greg", new Authorizations("X", "Y", "Z")); - assertTrue(c.securityOperations().getUserAuthorizations("greg").contains("X".getBytes())); - assertFalse(c.securityOperations().getUserAuthorizations("greg").contains("A".getBytes())); - } - - @Test - public void testBadMutations() throws Exception { - Connector c = new MockConnector("root", new MockInstance()); - c.tableOperations().create("test"); - BatchWriter bw = c - .createBatchWriter("test", new BatchWriterConfig().setMaxMemory(10000L).setMaxLatency(1000L, TimeUnit.MILLISECONDS).setMaxWriteThreads(4)); - - try { - bw.addMutation(null); - Assert.fail("addMutation should throw IAE for null mutation"); - } catch (IllegalArgumentException iae) {} - try { - bw.addMutations(null); - Assert.fail("addMutations should throw IAE for null iterable"); - } catch (IllegalArgumentException iae) {} - - bw.addMutations(Collections. emptyList()); - - Mutation bad = new Mutation("bad"); - try { - bw.addMutation(bad); - Assert.fail("addMutation should throw IAE for empty mutation"); - } catch (IllegalArgumentException iae) {} - - Mutation good = new Mutation("good"); - good.put(asText(random.nextInt()), asText(random.nextInt()), new Value("good".getBytes())); - List mutations = new ArrayList(); - mutations.add(good); - mutations.add(bad); - try { - bw.addMutations(mutations); - Assert.fail("addMutations should throw IAE if it contains empty mutation"); - } catch (IllegalArgumentException iae) {} - - bw.close(); - } - - @Test - public void testAggregation() throws Exception { - MockInstance mockInstance = new MockInstance(); - Connector c = mockInstance.getConnector("root", new PasswordToken("")); - String table = "perDayCounts"; - c.tableOperations().create(table); - IteratorSetting is = new IteratorSetting(10, "String Summation", SummingCombiner.class); - Combiner.setColumns(is, Collections.singletonList(new IteratorSetting.Column("day"))); - SummingCombiner.setEncodingType(is, SummingCombiner.Type.STRING); - c.tableOperations().attachIterator(table, is); - String keys[][] = { {"foo", "day", "20080101"}, {"foo", "day", "20080101"}, {"foo", "day", "20080103"}, {"bar", "day", "20080101"}, - {"bar", "day", "20080101"},}; - BatchWriter bw = c.createBatchWriter("perDayCounts", new BatchWriterConfig()); - for (String elt[] : keys) { - Mutation m = new Mutation(new Text(elt[0])); - m.put(new Text(elt[1]), new Text(elt[2]), new Value("1".getBytes())); - bw.addMutation(m); - } - bw.close(); - - Scanner s = c.createScanner("perDayCounts", Authorizations.EMPTY); - Iterator> iterator = s.iterator(); - assertTrue(iterator.hasNext()); - checkEntry(iterator.next(), "bar", "day", "20080101", "2"); - assertTrue(iterator.hasNext()); - checkEntry(iterator.next(), "foo", "day", "20080101", "2"); - assertTrue(iterator.hasNext()); - checkEntry(iterator.next(), "foo", "day", "20080103", "1"); - assertFalse(iterator.hasNext()); - } - - @Test - public void testDelete() throws Exception { - Connector c = new MockConnector("root", new MockInstance()); - c.tableOperations().create("test"); - BatchWriter bw = c.createBatchWriter("test", new BatchWriterConfig()); - - Mutation m1 = new Mutation("r1"); - - m1.put("cf1", "cq1", 1, "v1"); - - bw.addMutation(m1); - bw.flush(); - - Mutation m2 = new Mutation("r1"); - - m2.putDelete("cf1", "cq1", 2); - - bw.addMutation(m2); - bw.flush(); - - Scanner scanner = c.createScanner("test", Authorizations.EMPTY); - - int count = Iterators.size(scanner.iterator()); - - assertEquals(0, count); - - try { - c.tableOperations().create("test_this_$tableName"); - assertTrue(false); - - } catch (IllegalArgumentException iae) { - - } - } - - @Test - public void testDeletewithBatchDeleter() throws Exception { - Connector c = new MockConnector("root", new MockInstance()); - - // make sure we are using a clean table - if (c.tableOperations().exists("test")) - c.tableOperations().delete("test"); - c.tableOperations().create("test"); - - BatchDeleter deleter = c.createBatchDeleter("test", Authorizations.EMPTY, 2, new BatchWriterConfig()); - // first make sure it deletes fine when its empty - deleter.setRanges(Collections.singletonList(new Range(("r1")))); - deleter.delete(); - this.checkRemaining(c, "test", 0); - - // test deleting just one row - BatchWriter writer = c.createBatchWriter("test", new BatchWriterConfig()); - Mutation m = new Mutation("r1"); - m.put("fam", "qual", "value"); - writer.addMutation(m); - - // make sure the write goes through - writer.flush(); - writer.close(); - - deleter.setRanges(Collections.singletonList(new Range(("r1")))); - deleter.delete(); - this.checkRemaining(c, "test", 0); - - // test multi row deletes - writer = c.createBatchWriter("test", new BatchWriterConfig()); - m = new Mutation("r1"); - m.put("fam", "qual", "value"); - writer.addMutation(m); - Mutation m2 = new Mutation("r2"); - m2.put("fam", "qual", "value"); - writer.addMutation(m2); - - // make sure the write goes through - writer.flush(); - writer.close(); - - deleter.setRanges(Collections.singletonList(new Range(("r1")))); - deleter.delete(); - checkRemaining(c, "test", 1); - } - - /** - * Test to make sure that a certain number of rows remain - * - * @param c - * connector to the {@link MockInstance} - * @param tableName - * Table to check - * @param count - * number of entries to expect in the table - */ - private void checkRemaining(Connector c, String tableName, int count) throws Exception { - Scanner scanner = c.createScanner(tableName, Authorizations.EMPTY); - - int total = Iterators.size(scanner.iterator()); - assertEquals(count, total); - } - - @Test - public void testCMod() throws Exception { - // test writing to a table that the is being scanned - Connector c = new MockConnector("root", new MockInstance()); - c.tableOperations().create("test"); - BatchWriter bw = c.createBatchWriter("test", new BatchWriterConfig()); - - for (int i = 0; i < 10; i++) { - Mutation m1 = new Mutation("r" + i); - m1.put("cf1", "cq1", 1, "v" + i); - bw.addMutation(m1); - } - - bw.flush(); - - int count = 10; - - Scanner scanner = c.createScanner("test", Authorizations.EMPTY); - for (Entry entry : scanner) { - Key key = entry.getKey(); - Mutation m = new Mutation(key.getRow()); - m.put(key.getColumnFamily().toString(), key.getColumnQualifier().toString(), key.getTimestamp() + 1, "v" + (count)); - count++; - bw.addMutation(m); - } - - bw.flush(); - - count = 10; - - for (Entry entry : scanner) { - assertEquals(entry.getValue().toString(), "v" + (count++)); - } - - assertEquals(count, 20); - - try { - c.tableOperations().create("test_this_$tableName"); - assertTrue(false); - - } catch (IllegalArgumentException iae) { - - } - } - - private void checkEntry(Entry next, String row, String cf, String cq, String value) { - assertEquals(row, next.getKey().getRow().toString()); - assertEquals(cf, next.getKey().getColumnFamily().toString()); - assertEquals(cq, next.getKey().getColumnQualifier().toString()); - assertEquals(value, next.getValue().toString()); - } - - @Test - public void testMockMultiTableBatchWriter() throws Exception { - Connector c = new MockConnector("root", new MockInstance()); - c.tableOperations().create("a"); - c.tableOperations().create("b"); - MultiTableBatchWriter bw = c.createMultiTableBatchWriter(new BatchWriterConfig()); - Mutation m1 = new Mutation("r1"); - m1.put("cf1", "cq1", 1, "v1"); - BatchWriter b = bw.getBatchWriter("a"); - b.addMutation(m1); - b.flush(); - b = bw.getBatchWriter("b"); - b.addMutation(m1); - b.flush(); - - Scanner scanner = c.createScanner("a", Authorizations.EMPTY); - int count = Iterators.size(scanner.iterator()); - assertEquals(1, count); - scanner = c.createScanner("b", Authorizations.EMPTY); - count = Iterators.size(scanner.iterator()); - assertEquals(1, count); - - } - - @Test - public void testUpdate() throws Exception { - Connector c = new MockConnector("root", new MockInstance()); - c.tableOperations().create("test"); - BatchWriter bw = c.createBatchWriter("test", new BatchWriterConfig()); - - for (int i = 0; i < 10; i++) { - Mutation m = new Mutation("r1"); - m.put("cf1", "cq1", "" + i); - bw.addMutation(m); - } - - bw.close(); - - Scanner scanner = c.createScanner("test", Authorizations.EMPTY); - - Entry entry = scanner.iterator().next(); - - assertEquals("9", entry.getValue().toString()); - - } - - @Test - public void testMockConnectorReturnsCorrectInstance() throws AccumuloException, AccumuloSecurityException { - String name = "an-interesting-instance-name"; - Instance mockInstance = new MockInstance(name); - assertEquals(mockInstance, mockInstance.getConnector("foo", new PasswordToken("bar")).getInstance()); - assertEquals(name, mockInstance.getConnector("foo", new PasswordToken("bar")).getInstance().getInstanceName()); - } - -} http://git-wip-us.apache.org/repos/asf/accumulo/blob/b69291a3/core/src/test/java/org/apache/accumulo/core/client/mock/MockNamespacesTest.java ---------------------------------------------------------------------- diff --git a/core/src/test/java/org/apache/accumulo/core/client/mock/MockNamespacesTest.java b/core/src/test/java/org/apache/accumulo/core/client/mock/MockNamespacesTest.java deleted file mode 100644 index c1f39e2..0000000 --- a/core/src/test/java/org/apache/accumulo/core/client/mock/MockNamespacesTest.java +++ /dev/null @@ -1,297 +0,0 @@ -/* - * 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.mock; - -import static java.nio.charset.StandardCharsets.UTF_8; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; - -import java.util.EnumSet; -import java.util.HashSet; -import java.util.Map.Entry; - -import org.apache.accumulo.core.client.AccumuloException; -import org.apache.accumulo.core.client.AccumuloSecurityException; -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.IteratorSetting; -import org.apache.accumulo.core.client.NamespaceNotEmptyException; -import org.apache.accumulo.core.client.NamespaceNotFoundException; -import org.apache.accumulo.core.client.Scanner; -import org.apache.accumulo.core.client.TableNotFoundException; -import org.apache.accumulo.core.client.admin.NamespaceOperations; -import org.apache.accumulo.core.client.impl.Namespaces; -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.Value; -import org.apache.accumulo.core.iterators.Filter; -import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope; -import org.apache.accumulo.core.security.Authorizations; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TestName; - -@Deprecated -public class MockNamespacesTest { - - @Rule - public TestName test = new TestName(); - - private Connector conn; - - @Before - public void setupInstance() throws Exception { - Instance inst = new MockInstance(test.getMethodName()); - conn = inst.getConnector("user", new PasswordToken("pass")); - } - - /** - * This test creates a table without specifying a namespace. In this case, it puts the table into the default namespace. - */ - @Test - public void testDefaultNamespace() throws Exception { - String tableName = "test"; - - assertTrue(conn.namespaceOperations().exists(Namespaces.DEFAULT_NAMESPACE)); - conn.tableOperations().create(tableName); - assertTrue(conn.tableOperations().exists(tableName)); - } - - /** - * This test creates a new namespace "testing" and a table "testing.table1" which puts "table1" into the "testing" namespace. Then we create "testing.table2" - * which creates "table2" and puts it into "testing" as well. Then we make sure that you can't delete a namespace with tables in it, and then we delete the - * tables and delete the namespace. - */ - @Test - public void testCreateAndDeleteNamespace() throws Exception { - String namespace = "testing"; - String tableName1 = namespace + ".table1"; - String tableName2 = namespace + ".table2"; - - conn.namespaceOperations().create(namespace); - assertTrue(conn.namespaceOperations().exists(namespace)); - - conn.tableOperations().create(tableName1); - assertTrue(conn.tableOperations().exists(tableName1)); - - conn.tableOperations().create(tableName2); - assertTrue(conn.tableOperations().exists(tableName2)); - - // deleting - try { - // can't delete a namespace with tables in it - conn.namespaceOperations().delete(namespace); - fail(); - } catch (NamespaceNotEmptyException e) { - // ignore, supposed to happen - } - assertTrue(conn.namespaceOperations().exists(namespace)); - assertTrue(conn.tableOperations().exists(tableName1)); - assertTrue(conn.tableOperations().exists(tableName2)); - - conn.tableOperations().delete(tableName2); - assertTrue(!conn.tableOperations().exists(tableName2)); - assertTrue(conn.namespaceOperations().exists(namespace)); - - conn.tableOperations().delete(tableName1); - assertTrue(!conn.tableOperations().exists(tableName1)); - conn.namespaceOperations().delete(namespace); - assertTrue(!conn.namespaceOperations().exists(namespace)); - } - - /** - * This test creates a namespace, modifies it's properties, and checks to make sure that those properties are applied to its tables. To do something on a - * namespace-wide level, use {@link NamespaceOperations}. - * - * Checks to make sure namespace-level properties are overridden by table-level properties. - * - * Checks to see if the default namespace's properties work as well. - */ - - @Test - public void testNamespaceProperties() throws Exception { - String namespace = "propchange"; - String tableName1 = namespace + ".table1"; - String tableName2 = namespace + ".table2"; - - String propKey = Property.TABLE_SCAN_MAXMEM.getKey(); - String propVal = "42K"; - - conn.namespaceOperations().create(namespace); - conn.tableOperations().create(tableName1); - conn.namespaceOperations().setProperty(namespace, propKey, propVal); - - // check the namespace has the property - assertTrue(checkNamespaceHasProp(conn, namespace, propKey, propVal)); - - // check that the table gets it from the namespace - assertTrue(checkTableHasProp(conn, tableName1, propKey, propVal)); - - // test a second table to be sure the first wasn't magical - // (also, changed the order, the namespace has the property already) - conn.tableOperations().create(tableName2); - assertTrue(checkTableHasProp(conn, tableName2, propKey, propVal)); - - // test that table properties override namespace properties - String propKey2 = Property.TABLE_FILE_MAX.getKey(); - String propVal2 = "42"; - String tablePropVal = "13"; - - conn.tableOperations().setProperty(tableName2, propKey2, tablePropVal); - conn.namespaceOperations().setProperty("propchange", propKey2, propVal2); - - assertTrue(checkTableHasProp(conn, tableName2, propKey2, tablePropVal)); - - // now check that you can change the default namespace's properties - propVal = "13K"; - String tableName = "some_table"; - conn.tableOperations().create(tableName); - conn.namespaceOperations().setProperty(Namespaces.DEFAULT_NAMESPACE, propKey, propVal); - - assertTrue(checkTableHasProp(conn, tableName, propKey, propVal)); - - // test the properties server-side by configuring an iterator. - // should not show anything with column-family = 'a' - String tableName3 = namespace + ".table3"; - conn.tableOperations().create(tableName3); - - IteratorSetting setting = new IteratorSetting(250, "thing", SimpleFilter.class.getName()); - conn.namespaceOperations().attachIterator(namespace, setting); - - BatchWriter bw = conn.createBatchWriter(tableName3, new BatchWriterConfig()); - Mutation m = new Mutation("r"); - m.put("a", "b", new Value("abcde".getBytes())); - bw.addMutation(m); - bw.flush(); - bw.close(); - - // Scanner s = c.createScanner(tableName3, Authorizations.EMPTY); - // do scanners work correctly in mock? - // assertTrue(!s.iterator().hasNext()); - } - - /** - * This test renames and clones two separate table into different namespaces. different namespace. - */ - @Test - public void testRenameAndCloneTableToNewNamespace() throws Exception { - String namespace1 = "renamed"; - String namespace2 = "cloned"; - String tableName = "table"; - String tableName1 = "renamed.table1"; - // String tableName2 = "cloned.table2"; - - conn.tableOperations().create(tableName); - conn.namespaceOperations().create(namespace1); - conn.namespaceOperations().create(namespace2); - - conn.tableOperations().rename(tableName, tableName1); - - assertTrue(conn.tableOperations().exists(tableName1)); - assertTrue(!conn.tableOperations().exists(tableName)); - - // TODO implement clone in mock - // c.tableOperations().clone(tableName1, tableName2, false, null, null); - // assertTrue(c.tableOperations().exists(tableName1)); assertTrue(c.tableOperations().exists(tableName2)); - } - - /** - * This test renames a namespace and ensures that its tables are still correct - */ - @Test - public void testNamespaceRename() throws Exception { - String namespace1 = "n1"; - String namespace2 = "n2"; - String table = "t"; - - conn.namespaceOperations().create(namespace1); - conn.tableOperations().create(namespace1 + "." + table); - - conn.namespaceOperations().rename(namespace1, namespace2); - - assertTrue(!conn.namespaceOperations().exists(namespace1)); - assertTrue(conn.namespaceOperations().exists(namespace2)); - assertTrue(!conn.tableOperations().exists(namespace1 + "." + table)); - assertTrue(conn.tableOperations().exists(namespace2 + "." + table)); - } - - /** - * This tests adding iterators to a namespace, listing them, and removing them - */ - @Test - public void testNamespaceIterators() throws Exception { - String namespace = "iterator"; - String tableName = namespace + ".table"; - String iter = "thing"; - - conn.namespaceOperations().create(namespace); - conn.tableOperations().create(tableName); - - IteratorSetting setting = new IteratorSetting(250, iter, SimpleFilter.class.getName()); - HashSet scope = new HashSet(); - scope.add(IteratorScope.scan); - conn.namespaceOperations().attachIterator(namespace, setting, EnumSet.copyOf(scope)); - - BatchWriter bw = conn.createBatchWriter(tableName, new BatchWriterConfig()); - Mutation m = new Mutation("r"); - m.put("a", "b", new Value("abcde".getBytes(UTF_8))); - bw.addMutation(m); - bw.flush(); - - Scanner s = conn.createScanner(tableName, Authorizations.EMPTY); - System.out.println(s.iterator().next()); - // do scanners work correctly in mock? - // assertTrue(!s.iterator().hasNext()); - - assertTrue(conn.namespaceOperations().listIterators(namespace).containsKey(iter)); - conn.namespaceOperations().removeIterator(namespace, iter, EnumSet.copyOf(scope)); - } - - private boolean checkTableHasProp(Connector c, String t, String propKey, String propVal) throws AccumuloException, TableNotFoundException { - for (Entry e : c.tableOperations().getProperties(t)) { - if (e.getKey().equals(propKey) && e.getValue().equals(propVal)) { - return true; - } - } - return false; - } - - private boolean checkNamespaceHasProp(Connector c, String n, String propKey, String propVal) throws AccumuloException, NamespaceNotFoundException, - AccumuloSecurityException { - for (Entry e : c.namespaceOperations().getProperties(n)) { - if (e.getKey().equals(propKey) && e.getValue().equals(propVal)) { - return true; - } - } - return false; - } - - public static class SimpleFilter extends Filter { - @Override - public boolean accept(Key k, Value v) { - if (k.getColumnFamily().toString().equals("a")) - return false; - return true; - } - } -}