You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@accumulo.apache.org by GitBox <gi...@apache.org> on 2021/09/10 16:03:03 UTC

[GitHub] [accumulo] milleruntime opened a new pull request #2268: Clarify compact method

milleruntime opened a new pull request #2268:
URL: https://github.com/apache/accumulo/pull/2268


   * Make compact method in CompactableUtils just call compact and return
   the majc stats, removing the need to mutate the object across method calls. 
   * Pass the CompactionInfo object as a param to reduce the number of params 
   to the compact method. 
   * Pull logic from compact method and put into new bringOnline method, further 
   reducing the number of params to the compact method. The bringOnline method
   returns the completed StoredTabletFile 
   * Replace anonymous CompactionEnv classes with 2 new classes: MinCEnv & MajCEnv
   * Rename CompactionEnvironment to ExtCEnv to differentiate from the
   other types of CompactionEnv implementations
   * Add javadoc comments describing methods


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [accumulo] keith-turner commented on a change in pull request #2268: Clarify compact method

Posted by GitBox <gi...@apache.org>.
keith-turner commented on a change in pull request #2268:
URL: https://github.com/apache/accumulo/pull/2268#discussion_r706318697



##########
File path: server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/MajCEnv.java
##########
@@ -0,0 +1,96 @@
+/*
+ * 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.tserver.tablet;
+
+import org.apache.accumulo.core.conf.AccumuloConfiguration;
+import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.TableId;
+import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.core.iterators.IteratorUtil;
+import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
+import org.apache.accumulo.core.spi.compaction.CompactionKind;
+import org.apache.accumulo.core.tabletserver.thrift.TCompactionReason;
+import org.apache.accumulo.core.util.ratelimit.RateLimiter;
+import org.apache.accumulo.server.ServerContext;
+import org.apache.accumulo.server.compaction.FileCompactor;
+import org.apache.accumulo.server.iterators.SystemIteratorEnvironment;
+import org.apache.accumulo.server.iterators.TabletIteratorEnvironment;
+
+public class MajCEnv implements FileCompactor.CompactionEnv {
+  private final CompactionKind kind;
+  private final RateLimiter readLimiter;
+  private final RateLimiter writeLimiter;
+  private final boolean propagateDeletes;
+  private final boolean enabled;
+
+  public MajCEnv(CompactionKind kind, CompactableImpl.CompactionCheck compactionCheck,
+      RateLimiter readLimiter, RateLimiter writeLimiter, boolean propagateDeletes) {
+    this.kind = kind;
+    this.readLimiter = readLimiter;
+    this.writeLimiter = writeLimiter;
+    this.propagateDeletes = propagateDeletes;
+    this.enabled = compactionCheck.isCompactionEnabled();
+  }
+
+  @Override
+  public boolean isCompactionEnabled() {
+    return enabled;

Review comment:
       This should call `compactionCheck.isCompactionEnabled()` because running compactions call `isCompactionEnabled()` repeatedly to see if they should continue running.  That is how a running compaction is canceled.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [accumulo] keith-turner commented on a change in pull request #2268: Clarify compact method

Posted by GitBox <gi...@apache.org>.
keith-turner commented on a change in pull request #2268:
URL: https://github.com/apache/accumulo/pull/2268#discussion_r706318697



##########
File path: server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/MajCEnv.java
##########
@@ -0,0 +1,96 @@
+/*
+ * 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.tserver.tablet;
+
+import org.apache.accumulo.core.conf.AccumuloConfiguration;
+import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.TableId;
+import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.core.iterators.IteratorUtil;
+import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
+import org.apache.accumulo.core.spi.compaction.CompactionKind;
+import org.apache.accumulo.core.tabletserver.thrift.TCompactionReason;
+import org.apache.accumulo.core.util.ratelimit.RateLimiter;
+import org.apache.accumulo.server.ServerContext;
+import org.apache.accumulo.server.compaction.FileCompactor;
+import org.apache.accumulo.server.iterators.SystemIteratorEnvironment;
+import org.apache.accumulo.server.iterators.TabletIteratorEnvironment;
+
+public class MajCEnv implements FileCompactor.CompactionEnv {
+  private final CompactionKind kind;
+  private final RateLimiter readLimiter;
+  private final RateLimiter writeLimiter;
+  private final boolean propagateDeletes;
+  private final boolean enabled;
+
+  public MajCEnv(CompactionKind kind, CompactableImpl.CompactionCheck compactionCheck,
+      RateLimiter readLimiter, RateLimiter writeLimiter, boolean propagateDeletes) {
+    this.kind = kind;
+    this.readLimiter = readLimiter;
+    this.writeLimiter = writeLimiter;
+    this.propagateDeletes = propagateDeletes;
+    this.enabled = compactionCheck.isCompactionEnabled();
+  }
+
+  @Override
+  public boolean isCompactionEnabled() {
+    return enabled;

Review comment:
       This should call `compactionCheck.isCompactionEnabled()` because running compactions call this repeatedly to see if they should continue running.  That is how a running compaction is canceled.

##########
File path: server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/CompactableUtils.java
##########
@@ -552,88 +544,42 @@ private static AccumuloConfiguration getCompactionConfig(TableConfiguration tabl
     return copy;
   }
 
-  static StoredTabletFile compact(Tablet tablet, CompactionJob job, Set<StoredTabletFile> jobFiles,
-      Long compactionId, Set<StoredTabletFile> selectedFiles, boolean propagateDeletes,
-      CompactableImpl.CompactionHelper helper, List<IteratorSetting> iters,
-      CompactionCheck compactionCheck, RateLimiter readLimiter, RateLimiter writeLimiter,
-      CompactionStats stats) throws IOException, CompactionCanceledException {
-    StoredTabletFile metaFile;
-    CompactionEnv cenv = new CompactionEnv() {
-      @Override
-      public boolean isCompactionEnabled() {
-        return compactionCheck.isCompactionEnabled();
-      }
-
-      @Override
-      public IteratorScope getIteratorScope() {
-        return IteratorScope.majc;
-      }
-
-      @Override
-      public RateLimiter getReadLimiter() {
-        return readLimiter;
-      }
-
-      @Override
-      public RateLimiter getWriteLimiter() {
-        return writeLimiter;
-      }
-
-      @Override
-      public SystemIteratorEnvironment createIteratorEnv(ServerContext context,
-          AccumuloConfiguration acuTableConf, TableId tableId) {
-        return new TabletIteratorEnvironment(context, IteratorScope.majc, !propagateDeletes,
-            acuTableConf, tableId, job.getKind());
-      }
-
-      @Override
-      public SortedKeyValueIterator<Key,Value> getMinCIterator() {
-        throw new UnsupportedOperationException();
-      }
-
-      @Override
-      public TCompactionReason getReason() {
-        switch (job.getKind()) {
-          case USER:
-            return TCompactionReason.USER;
-          case CHOP:
-            return TCompactionReason.CHOP;
-          case SELECTOR:
-          case SYSTEM:
-          default:
-            return TCompactionReason.SYSTEM;
-        }
-      }
-    };
+  /**
+   * Create the FileCompactor and finally call compact. Returns the Major CompactionStats.
+   */
+  static CompactionStats compact(Tablet tablet, CompactionJob job,
+      CompactableImpl.CompactionInfo cInfo, CompactionEnv cenv,
+      Map<StoredTabletFile,DataFileValue> compactFiles, TabletFile tmpFileName)
+      throws IOException, CompactionCanceledException {
+    boolean propagateDeletes = cInfo.propagateDeletes;

Review comment:
       Could this be inlined?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [accumulo] milleruntime merged pull request #2268: Clarify compact method

Posted by GitBox <gi...@apache.org>.
milleruntime merged pull request #2268:
URL: https://github.com/apache/accumulo/pull/2268


   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org