[GitHub] [hbase] wchevreuil commented on a change in pull request #3488: HBASE-25393 Support split and merge region with direct insert into CF…

2021-08-03 Thread GitBox


wchevreuil commented on a change in pull request #3488:
URL: https://github.com/apache/hbase/pull/3488#discussion_r682092734



##
File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/DirectStoreMergeRegionsStrategy.java
##
@@ -0,0 +1,105 @@
+/**
+ * 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.hadoop.hbase.master.assignment;
+
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.TableDescriptor;
+import org.apache.hadoop.hbase.io.hfile.CacheConfig;
+import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
+import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
+import org.apache.hadoop.hbase.regionserver.HStoreFile;
+import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
+import org.apache.yetus.audience.InterfaceAudience;
+
+import java.io.IOException;
+import java.util.Collection;
+
+/**
+ * MergeRegionStrategy implementation to be used in 
combination with
+ * PersistedStoreEngine to avoid renames when merging regions.
+ *
+ * To use it, define the following properties under master configuration:

Review comment:
   Changed to load both mergestrategy and write strategy from the table 
config.




-- 
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: issues-unsubscr...@hbase.apache.org

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




[GitHub] [hbase] wchevreuil commented on a change in pull request #3488: HBASE-25393 Support split and merge region with direct insert into CF…

2021-07-23 Thread GitBox


wchevreuil commented on a change in pull request #3488:
URL: https://github.com/apache/hbase/pull/3488#discussion_r675663329



##
File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DirectStoreFSWriteStrategy.java
##
@@ -0,0 +1,258 @@
+/**
+ * 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.hadoop.hbase.regionserver;
+
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.yetus.audience.InterfaceAudience;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * HRegionFileSystemWriteStrategy implementation to be used in 
combination with
+ * PersistedStoreEngine to avoid renames when splitting and 
merging regions.
+ *
+ * To use it, define the following properties under master configuration:
+ * 1) 
+ *  hbase.hregion.file.write.strategy
+ *  
org.apache.hadoop.hbase.regionserver.DirectStoreFSWriteStrategy
+ *
+ * 2) 
+ *  hbase.hregion.merge.strategy
+ *  
org.apache.hadoop.hbase.master.assignment.DirectStoreMergeRegionsStrategy
+ *
+ *
+ * This will create the resulting merging and splitting regions directory 
straight under
+ * the table dir, instead of creating it under the temporary ".tmp" or 
".merges" dirs,
+ * as done by the default implementation.
+ */
+@InterfaceAudience.Private
+public class DirectStoreFSWriteStrategy extends HRegionFileSystemWriteStrategy 
{
+  private StoreFilePathAccessor accessor;
+  private Map>> regionSplitReferences = new 
ConcurrentHashMap<>();
+  private Map> mergeReferences = new HashMap();
+
+  public DirectStoreFSWriteStrategy(HRegionFileSystem fileSystem) throws 
IOException {
+super(fileSystem);
+this.accessor =  
StoreFileTrackingUtils.createStoreFilePathAccessor(fileSystem.conf,
+  ConnectionFactory.createConnection(fileSystem.conf));
+  }
+
+  /**
+   * The parent directory where to create the splits dirs is
+   * the table directory itself, in this case.
+   * @return Path representing the table directory.
+   */
+  @Override
+  public Path getParentSplitsDir() {
+return fileSystem.getTableDir();
+  }
+
+  /**
+   * The parent directory where to create the merge dir is
+   * the table directory itself, in this case.
+   * @return Path representing the table directory.
+   */
+  @Override
+  public Path getParentMergesDir() {
+return fileSystem.getTableDir();
+  }
+
+  /**
+   * Creates the directories for the respective split daughters directly under 
the
+   * table directory, instead of default behaviour of doing it under temp 
dirs, initially.
+   * @param daughterA the first half of the split region
+   * @param daughterB the second half of the split region
+   *
+   * @throws IOException if directories creation fails.
+   */
+  @Override
+  public void createSplitsDir(RegionInfo daughterA, RegionInfo daughterB)
+throws IOException {
+Path splitdir = getParentSplitsDir();
+// splitDir doesn't exists now. No need to do an exists() call for it.
+if (!fileSystem.getFileSystem().exists(splitdir)) {
+  throw new IOException("Table dir for splitting region not found:  " + 
splitdir);
+}
+Path daughterADir = getSplitsDir(daughterA);
+if (!fileSystem.createDir(daughterADir)) {
+  throw new IOException("Failed create of " + daughterADir);
+}
+Path daughterBDir = getSplitsDir(daughterB);
+if (!fileSystem.createDir(daughterBDir)) {
+  throw new IOException("Failed create of " + daughterBDir);
+}
+  }
+
+  /**
+   * Just validates that merges parent, the actual table dir in this case, 
exists.
+   * @throws IOException if table dir doesn't exist.
+   */
+  @Override
+  public void createMergesDir() throws IOException {
+//When writing directly, avoiding renames, merges parent is the table dir 
itself, so it
+// should exist already, so just validate it exist then do nothing
+Path mergesdir = getParentMergesDir();
+if (!fileSystem.fs.exists(mergesdir)) {
+ 

[GitHub] [hbase] wchevreuil commented on a change in pull request #3488: HBASE-25393 Support split and merge region with direct insert into CF…

2021-07-23 Thread GitBox


wchevreuil commented on a change in pull request #3488:
URL: https://github.com/apache/hbase/pull/3488#discussion_r675663329



##
File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DirectStoreFSWriteStrategy.java
##
@@ -0,0 +1,258 @@
+/**
+ * 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.hadoop.hbase.regionserver;
+
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.yetus.audience.InterfaceAudience;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * HRegionFileSystemWriteStrategy implementation to be used in 
combination with
+ * PersistedStoreEngine to avoid renames when splitting and 
merging regions.
+ *
+ * To use it, define the following properties under master configuration:
+ * 1) 
+ *  hbase.hregion.file.write.strategy
+ *  
org.apache.hadoop.hbase.regionserver.DirectStoreFSWriteStrategy
+ *
+ * 2) 
+ *  hbase.hregion.merge.strategy
+ *  
org.apache.hadoop.hbase.master.assignment.DirectStoreMergeRegionsStrategy
+ *
+ *
+ * This will create the resulting merging and splitting regions directory 
straight under
+ * the table dir, instead of creating it under the temporary ".tmp" or 
".merges" dirs,
+ * as done by the default implementation.
+ */
+@InterfaceAudience.Private
+public class DirectStoreFSWriteStrategy extends HRegionFileSystemWriteStrategy 
{
+  private StoreFilePathAccessor accessor;
+  private Map>> regionSplitReferences = new 
ConcurrentHashMap<>();
+  private Map> mergeReferences = new HashMap();
+
+  public DirectStoreFSWriteStrategy(HRegionFileSystem fileSystem) throws 
IOException {
+super(fileSystem);
+this.accessor =  
StoreFileTrackingUtils.createStoreFilePathAccessor(fileSystem.conf,
+  ConnectionFactory.createConnection(fileSystem.conf));
+  }
+
+  /**
+   * The parent directory where to create the splits dirs is
+   * the table directory itself, in this case.
+   * @return Path representing the table directory.
+   */
+  @Override
+  public Path getParentSplitsDir() {
+return fileSystem.getTableDir();
+  }
+
+  /**
+   * The parent directory where to create the merge dir is
+   * the table directory itself, in this case.
+   * @return Path representing the table directory.
+   */
+  @Override
+  public Path getParentMergesDir() {
+return fileSystem.getTableDir();
+  }
+
+  /**
+   * Creates the directories for the respective split daughters directly under 
the
+   * table directory, instead of default behaviour of doing it under temp 
dirs, initially.
+   * @param daughterA the first half of the split region
+   * @param daughterB the second half of the split region
+   *
+   * @throws IOException if directories creation fails.
+   */
+  @Override
+  public void createSplitsDir(RegionInfo daughterA, RegionInfo daughterB)
+throws IOException {
+Path splitdir = getParentSplitsDir();
+// splitDir doesn't exists now. No need to do an exists() call for it.
+if (!fileSystem.getFileSystem().exists(splitdir)) {
+  throw new IOException("Table dir for splitting region not found:  " + 
splitdir);
+}
+Path daughterADir = getSplitsDir(daughterA);
+if (!fileSystem.createDir(daughterADir)) {
+  throw new IOException("Failed create of " + daughterADir);
+}
+Path daughterBDir = getSplitsDir(daughterB);
+if (!fileSystem.createDir(daughterBDir)) {
+  throw new IOException("Failed create of " + daughterBDir);
+}
+  }
+
+  /**
+   * Just validates that merges parent, the actual table dir in this case, 
exists.
+   * @throws IOException if table dir doesn't exist.
+   */
+  @Override
+  public void createMergesDir() throws IOException {
+//When writing directly, avoiding renames, merges parent is the table dir 
itself, so it
+// should exist already, so just validate it exist then do nothing
+Path mergesdir = getParentMergesDir();
+if (!fileSystem.fs.exists(mergesdir)) {
+ 

[GitHub] [hbase] wchevreuil commented on a change in pull request #3488: HBASE-25393 Support split and merge region with direct insert into CF…

2021-07-23 Thread GitBox


wchevreuil commented on a change in pull request #3488:
URL: https://github.com/apache/hbase/pull/3488#discussion_r675621273



##
File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionFileSystemWriteStrategy.java
##
@@ -0,0 +1,220 @@
+/**
+ * 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.hadoop.hbase.regionserver;
+
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.PrivateCellUtil;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.io.Reference;
+import org.apache.yetus.audience.InterfaceAudience;
+
+import java.io.IOException;
+import java.util.Optional;
+
+/**
+ * HRegionFileSystem write strategy to decouple splits/merge create dir and 
commit logic
+ * from HRegionFileSystem, allowing for a plugable behaviour.
+ */
+@InterfaceAudience.Private
+public abstract class HRegionFileSystemWriteStrategy {
+
+  protected HRegionFileSystem fileSystem;
+
+  public HRegionFileSystemWriteStrategy(HRegionFileSystem fileSystem){
+this.fileSystem = fileSystem;
+  }
+
+  /**
+   * Returns the directory Path for the region split resulting daughter.
+   * @param hri for the split resulting daughter region.
+   * @return a path under tmp dir for the resulting daughter region.
+   */
+  public Path getSplitsDir(final RegionInfo hri) {
+return new Path(getParentSplitsDir(), hri.getEncodedName());
+  }
+
+  /**
+   * Defines the parent dir for the split dir.
+   * @return
+   */
+  public abstract Path getParentSplitsDir();
+
+  /**
+   * Defines the parent dir for the merges dir.
+   * @return
+   */
+  public abstract Path getParentMergesDir();

Review comment:
   > I think it might be cleaner to push the RegionInfo into this call. Let 
the implementation decide where to put the "region dir" it's creating.
   
   So this is called by `HRegionFileSystem.getMergesDir`. In the original 
implementation, what `HRegionFileSystem.getMergesDir` returns is a parent dir 
of the result merging region dir. Implementations are deciding where result 
merging region dir is gonna be created (`TBL_DIR/R1/.merges` for the default 
strategy, `TBL_DIR/` for the direct store one).
   
   > The "getParent..." terminology is a little bit confusing to me. I think I 
really just want to know if I'm splitting or merging a region, where does the 
new region(s) get created?
   
   :) Eh, I got confused by the original naming. When looking at 
`HRegionFileSystem.getMergesDir`, `HRegionFileSystem.getSplitsDir`, I was 
expecting those to already give me the merge/splits paths, not just the parents 
where those were to be created. I can rename these methods accordingly, though.




-- 
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: issues-unsubscr...@hbase.apache.org

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




[GitHub] [hbase] wchevreuil commented on a change in pull request #3488: HBASE-25393 Support split and merge region with direct insert into CF…

2021-07-23 Thread GitBox


wchevreuil commented on a change in pull request #3488:
URL: https://github.com/apache/hbase/pull/3488#discussion_r675621273



##
File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionFileSystemWriteStrategy.java
##
@@ -0,0 +1,220 @@
+/**
+ * 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.hadoop.hbase.regionserver;
+
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.PrivateCellUtil;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.io.Reference;
+import org.apache.yetus.audience.InterfaceAudience;
+
+import java.io.IOException;
+import java.util.Optional;
+
+/**
+ * HRegionFileSystem write strategy to decouple splits/merge create dir and 
commit logic
+ * from HRegionFileSystem, allowing for a plugable behaviour.
+ */
+@InterfaceAudience.Private
+public abstract class HRegionFileSystemWriteStrategy {
+
+  protected HRegionFileSystem fileSystem;
+
+  public HRegionFileSystemWriteStrategy(HRegionFileSystem fileSystem){
+this.fileSystem = fileSystem;
+  }
+
+  /**
+   * Returns the directory Path for the region split resulting daughter.
+   * @param hri for the split resulting daughter region.
+   * @return a path under tmp dir for the resulting daughter region.
+   */
+  public Path getSplitsDir(final RegionInfo hri) {
+return new Path(getParentSplitsDir(), hri.getEncodedName());
+  }
+
+  /**
+   * Defines the parent dir for the split dir.
+   * @return
+   */
+  public abstract Path getParentSplitsDir();
+
+  /**
+   * Defines the parent dir for the merges dir.
+   * @return
+   */
+  public abstract Path getParentMergesDir();

Review comment:
   > I think it might be cleaner to push the RegionInfo into this call. Let 
the implementation decide where to put the "region dir" it's creating.
   
   So this is called by `HRegionFileSystem.getMergesDir`. In the original 
implementation, what `HRegionFileSystem.getMergesDir` returns is a parent dir 
of the result merging region dir. Implementations are deciding where result 
merging region dir is gonna be created (`TBL_DIR/R1/.merges` for the default 
strategy, `TBL_DIR/` for the direct store one).
   
   > The "getParent..." terminology is a little bit confusing to me. I think I 
really just want to know if I'm splitting or merging a region, where does the 
new region(s) get created?
   
   :) Eh, I got confused by the original naming. When looking at 
`HRegionFileSystem.getMergesDir`, ``HRegionFileSystem.getSplitsDir`, I was 
expecting those to already give me the merge/splits paths, not just the parents 
where those were to be created. I can rename these methods accordingly, though.




-- 
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: issues-unsubscr...@hbase.apache.org

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




[GitHub] [hbase] wchevreuil commented on a change in pull request #3488: HBASE-25393 Support split and merge region with direct insert into CF…

2021-07-23 Thread GitBox


wchevreuil commented on a change in pull request #3488:
URL: https://github.com/apache/hbase/pull/3488#discussion_r675608482



##
File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DirectStoreFSWriteStrategy.java
##
@@ -0,0 +1,258 @@
+/**
+ * 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.hadoop.hbase.regionserver;
+
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.yetus.audience.InterfaceAudience;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * HRegionFileSystemWriteStrategy implementation to be used in 
combination with
+ * PersistedStoreEngine to avoid renames when splitting and 
merging regions.
+ *
+ * To use it, define the following properties under master configuration:
+ * 1) 
+ *  hbase.hregion.file.write.strategy
+ *  
org.apache.hadoop.hbase.regionserver.DirectStoreFSWriteStrategy
+ *
+ * 2) 
+ *  hbase.hregion.merge.strategy
+ *  
org.apache.hadoop.hbase.master.assignment.DirectStoreMergeRegionsStrategy
+ *
+ *
+ * This will create the resulting merging and splitting regions directory 
straight under
+ * the table dir, instead of creating it under the temporary ".tmp" or 
".merges" dirs,
+ * as done by the default implementation.
+ */
+@InterfaceAudience.Private
+public class DirectStoreFSWriteStrategy extends HRegionFileSystemWriteStrategy 
{
+  private StoreFilePathAccessor accessor;
+  private Map>> regionSplitReferences = new 
ConcurrentHashMap<>();
+  private Map> mergeReferences = new HashMap();
+
+  public DirectStoreFSWriteStrategy(HRegionFileSystem fileSystem) throws 
IOException {
+super(fileSystem);
+this.accessor =  
StoreFileTrackingUtils.createStoreFilePathAccessor(fileSystem.conf,
+  ConnectionFactory.createConnection(fileSystem.conf));
+  }
+
+  /**
+   * The parent directory where to create the splits dirs is
+   * the table directory itself, in this case.
+   * @return Path representing the table directory.
+   */
+  @Override
+  public Path getParentSplitsDir() {
+return fileSystem.getTableDir();
+  }
+
+  /**
+   * The parent directory where to create the merge dir is
+   * the table directory itself, in this case.
+   * @return Path representing the table directory.
+   */
+  @Override
+  public Path getParentMergesDir() {
+return fileSystem.getTableDir();
+  }
+
+  /**
+   * Creates the directories for the respective split daughters directly under 
the
+   * table directory, instead of default behaviour of doing it under temp 
dirs, initially.
+   * @param daughterA the first half of the split region
+   * @param daughterB the second half of the split region
+   *
+   * @throws IOException if directories creation fails.
+   */
+  @Override
+  public void createSplitsDir(RegionInfo daughterA, RegionInfo daughterB)
+throws IOException {
+Path splitdir = getParentSplitsDir();
+// splitDir doesn't exists now. No need to do an exists() call for it.
+if (!fileSystem.getFileSystem().exists(splitdir)) {
+  throw new IOException("Table dir for splitting region not found:  " + 
splitdir);
+}
+Path daughterADir = getSplitsDir(daughterA);
+if (!fileSystem.createDir(daughterADir)) {
+  throw new IOException("Failed create of " + daughterADir);
+}
+Path daughterBDir = getSplitsDir(daughterB);
+if (!fileSystem.createDir(daughterBDir)) {
+  throw new IOException("Failed create of " + daughterBDir);
+}
+  }
+
+  /**
+   * Just validates that merges parent, the actual table dir in this case, 
exists.
+   * @throws IOException if table dir doesn't exist.
+   */
+  @Override
+  public void createMergesDir() throws IOException {
+//When writing directly, avoiding renames, merges parent is the table dir 
itself, so it
+// should exist already, so just validate it exist then do nothing
+Path mergesdir = getParentMergesDir();
+if (!fileSystem.fs.exists(mergesdir)) {
+ 

[GitHub] [hbase] wchevreuil commented on a change in pull request #3488: HBASE-25393 Support split and merge region with direct insert into CF…

2021-07-23 Thread GitBox


wchevreuil commented on a change in pull request #3488:
URL: https://github.com/apache/hbase/pull/3488#discussion_r675586579



##
File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DirectStoreFSWriteStrategy.java
##
@@ -0,0 +1,258 @@
+/**
+ * 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.hadoop.hbase.regionserver;
+
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.yetus.audience.InterfaceAudience;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * HRegionFileSystemWriteStrategy implementation to be used in 
combination with
+ * PersistedStoreEngine to avoid renames when splitting and 
merging regions.
+ *
+ * To use it, define the following properties under master configuration:
+ * 1) 
+ *  hbase.hregion.file.write.strategy
+ *  
org.apache.hadoop.hbase.regionserver.DirectStoreFSWriteStrategy
+ *
+ * 2) 
+ *  hbase.hregion.merge.strategy
+ *  
org.apache.hadoop.hbase.master.assignment.DirectStoreMergeRegionsStrategy
+ *
+ *
+ * This will create the resulting merging and splitting regions directory 
straight under
+ * the table dir, instead of creating it under the temporary ".tmp" or 
".merges" dirs,
+ * as done by the default implementation.
+ */
+@InterfaceAudience.Private
+public class DirectStoreFSWriteStrategy extends HRegionFileSystemWriteStrategy 
{
+  private StoreFilePathAccessor accessor;
+  private Map>> regionSplitReferences = new 
ConcurrentHashMap<>();
+  private Map> mergeReferences = new HashMap();
+
+  public DirectStoreFSWriteStrategy(HRegionFileSystem fileSystem) throws 
IOException {
+super(fileSystem);
+this.accessor =  
StoreFileTrackingUtils.createStoreFilePathAccessor(fileSystem.conf,
+  ConnectionFactory.createConnection(fileSystem.conf));
+  }
+
+  /**
+   * The parent directory where to create the splits dirs is
+   * the table directory itself, in this case.
+   * @return Path representing the table directory.
+   */
+  @Override
+  public Path getParentSplitsDir() {
+return fileSystem.getTableDir();
+  }
+
+  /**
+   * The parent directory where to create the merge dir is
+   * the table directory itself, in this case.
+   * @return Path representing the table directory.
+   */
+  @Override
+  public Path getParentMergesDir() {
+return fileSystem.getTableDir();
+  }
+
+  /**
+   * Creates the directories for the respective split daughters directly under 
the
+   * table directory, instead of default behaviour of doing it under temp 
dirs, initially.
+   * @param daughterA the first half of the split region
+   * @param daughterB the second half of the split region
+   *
+   * @throws IOException if directories creation fails.
+   */
+  @Override
+  public void createSplitsDir(RegionInfo daughterA, RegionInfo daughterB)
+throws IOException {
+Path splitdir = getParentSplitsDir();
+// splitDir doesn't exists now. No need to do an exists() call for it.
+if (!fileSystem.getFileSystem().exists(splitdir)) {
+  throw new IOException("Table dir for splitting region not found:  " + 
splitdir);
+}
+Path daughterADir = getSplitsDir(daughterA);
+if (!fileSystem.createDir(daughterADir)) {
+  throw new IOException("Failed create of " + daughterADir);
+}
+Path daughterBDir = getSplitsDir(daughterB);
+if (!fileSystem.createDir(daughterBDir)) {
+  throw new IOException("Failed create of " + daughterBDir);
+}
+  }
+
+  /**
+   * Just validates that merges parent, the actual table dir in this case, 
exists.
+   * @throws IOException if table dir doesn't exist.
+   */
+  @Override
+  public void createMergesDir() throws IOException {
+//When writing directly, avoiding renames, merges parent is the table dir 
itself, so it
+// should exist already, so just validate it exist then do nothing
+Path mergesdir = getParentMergesDir();
+if (!fileSystem.fs.exists(mergesdir)) {
+ 

[GitHub] [hbase] wchevreuil commented on a change in pull request #3488: HBASE-25393 Support split and merge region with direct insert into CF…

2021-07-22 Thread GitBox


wchevreuil commented on a change in pull request #3488:
URL: https://github.com/apache/hbase/pull/3488#discussion_r675142140



##
File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionFileSystem.java
##
@@ -1247,4 +1149,186 @@ private static void sleepBeforeRetry(String msg, int 
sleepMultiplier, int baseSl
 }
 Thread.sleep((long)baseSleepBeforeRetries * sleepMultiplier);
   }
+
+  private void createWriteStrategy(Configuration conf) {
+String className = conf.get(REGION_WRITE_STRATEGY, 
DefaultWriteStrategy.class.getName());
+try {
+  LOG.info("instantiating write strategy {}", className);
+  writeStrategy = ReflectionUtils.instantiateWithCustomCtor(className,
+new Class[] { HRegionFileSystem.class }, new Object[] { this });
+} catch (Exception e) {
+  LOG.error("Unable to create write strategy: {}", className, e);
+}
+  }
+
+  public static class DefaultWriteStrategy extends 
HRegionFileSystemWriteStrategy {
+
+public DefaultWriteStrategy(HRegionFileSystem fileSystem){
+  super(fileSystem);
+}
+
+/**
+ * Constructs a Path for the split dir as follows:
+ *  "/hbase/data/NS/TABLE/PARENT_REGION_DIR/.splits/"
+ * @return the temporary parent path for the split dir
+ */
+@Override
+public Path getParentSplitsDir() {
+  return new Path(fileSystem.getRegionDir(), REGION_SPLITS_DIR);
+}
+
+/**
+ * Constructs a Path for the merged dir as follows:
+ *  "/hbase/data/NS/TABLE/PARENT_REGION_DIR/.merges/"
+ * @return the temporary parent path for the merges dir.
+ */
+@Override
+public Path getParentMergesDir() {
+  return new Path(fileSystem.getRegionDir(), REGION_MERGES_DIR);
+}
+
+/**
+ * Creates the region splits directory. Assumes getSplitsDir 
implementation returns a tmp dir,
+ * therefore, it deletes any existing directory returned by getSplitsDir.
+ *
+ * @param daughterA the first half of the split region
+ * @param daughterB the second half of the split region
+ *
+ * @throws IOException if splits dir creation fails.
+ */
+@Override
+public void createSplitsDir(RegionInfo daughterA, RegionInfo daughterB) 
throws IOException {
+  Path splitdir = getParentSplitsDir();
+  if (this.fileSystem.fs.exists(splitdir)) {
+LOG.info("The " + splitdir + " directory exists.  Hence deleting it to 
recreate it");

Review comment:
   Ack




-- 
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: issues-unsubscr...@hbase.apache.org

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




[GitHub] [hbase] wchevreuil commented on a change in pull request #3488: HBASE-25393 Support split and merge region with direct insert into CF…

2021-07-22 Thread GitBox


wchevreuil commented on a change in pull request #3488:
URL: https://github.com/apache/hbase/pull/3488#discussion_r675141684



##
File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionFileSystem.java
##
@@ -1247,4 +1149,186 @@ private static void sleepBeforeRetry(String msg, int 
sleepMultiplier, int baseSl
 }
 Thread.sleep((long)baseSleepBeforeRetries * sleepMultiplier);
   }
+
+  private void createWriteStrategy(Configuration conf) {
+String className = conf.get(REGION_WRITE_STRATEGY, 
DefaultWriteStrategy.class.getName());
+try {
+  LOG.info("instantiating write strategy {}", className);
+  writeStrategy = ReflectionUtils.instantiateWithCustomCtor(className,
+new Class[] { HRegionFileSystem.class }, new Object[] { this });
+} catch (Exception e) {
+  LOG.error("Unable to create write strategy: {}", className, e);

Review comment:
   Ack




-- 
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: issues-unsubscr...@hbase.apache.org

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




[GitHub] [hbase] wchevreuil commented on a change in pull request #3488: HBASE-25393 Support split and merge region with direct insert into CF…

2021-07-22 Thread GitBox


wchevreuil commented on a change in pull request #3488:
URL: https://github.com/apache/hbase/pull/3488#discussion_r675107534



##
File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DirectStoreFSWriteStrategy.java
##
@@ -0,0 +1,258 @@
+/**
+ * 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.hadoop.hbase.regionserver;
+
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.yetus.audience.InterfaceAudience;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * HRegionFileSystemWriteStrategy implementation to be used in 
combination with
+ * PersistedStoreEngine to avoid renames when splitting and 
merging regions.
+ *
+ * To use it, define the following properties under master configuration:
+ * 1) 
+ *  hbase.hregion.file.write.strategy
+ *  
org.apache.hadoop.hbase.regionserver.DirectStoreFSWriteStrategy
+ *
+ * 2) 
+ *  hbase.hregion.merge.strategy
+ *  
org.apache.hadoop.hbase.master.assignment.DirectStoreMergeRegionsStrategy
+ *
+ *
+ * This will create the resulting merging and splitting regions directory 
straight under
+ * the table dir, instead of creating it under the temporary ".tmp" or 
".merges" dirs,
+ * as done by the default implementation.
+ */
+@InterfaceAudience.Private
+public class DirectStoreFSWriteStrategy extends HRegionFileSystemWriteStrategy 
{
+  private StoreFilePathAccessor accessor;
+  private Map>> regionSplitReferences = new 
ConcurrentHashMap<>();
+  private Map> mergeReferences = new HashMap();
+
+  public DirectStoreFSWriteStrategy(HRegionFileSystem fileSystem) throws 
IOException {
+super(fileSystem);
+this.accessor =  
StoreFileTrackingUtils.createStoreFilePathAccessor(fileSystem.conf,
+  ConnectionFactory.createConnection(fileSystem.conf));
+  }
+
+  /**
+   * The parent directory where to create the splits dirs is
+   * the table directory itself, in this case.
+   * @return Path representing the table directory.
+   */
+  @Override
+  public Path getParentSplitsDir() {
+return fileSystem.getTableDir();
+  }
+
+  /**
+   * The parent directory where to create the merge dir is
+   * the table directory itself, in this case.
+   * @return Path representing the table directory.
+   */
+  @Override
+  public Path getParentMergesDir() {
+return fileSystem.getTableDir();
+  }
+
+  /**
+   * Creates the directories for the respective split daughters directly under 
the
+   * table directory, instead of default behaviour of doing it under temp 
dirs, initially.
+   * @param daughterA the first half of the split region
+   * @param daughterB the second half of the split region
+   *
+   * @throws IOException if directories creation fails.
+   */
+  @Override
+  public void createSplitsDir(RegionInfo daughterA, RegionInfo daughterB)
+throws IOException {
+Path splitdir = getParentSplitsDir();
+// splitDir doesn't exists now. No need to do an exists() call for it.
+if (!fileSystem.getFileSystem().exists(splitdir)) {
+  throw new IOException("Table dir for splitting region not found:  " + 
splitdir);
+}
+Path daughterADir = getSplitsDir(daughterA);
+if (!fileSystem.createDir(daughterADir)) {
+  throw new IOException("Failed create of " + daughterADir);
+}
+Path daughterBDir = getSplitsDir(daughterB);
+if (!fileSystem.createDir(daughterBDir)) {
+  throw new IOException("Failed create of " + daughterBDir);
+}
+  }
+
+  /**
+   * Just validates that merges parent, the actual table dir in this case, 
exists.
+   * @throws IOException if table dir doesn't exist.
+   */
+  @Override
+  public void createMergesDir() throws IOException {
+//When writing directly, avoiding renames, merges parent is the table dir 
itself, so it
+// should exist already, so just validate it exist then do nothing
+Path mergesdir = getParentMergesDir();
+if (!fileSystem.fs.exists(mergesdir)) {
+ 

[GitHub] [hbase] wchevreuil commented on a change in pull request #3488: HBASE-25393 Support split and merge region with direct insert into CF…

2021-07-22 Thread GitBox


wchevreuil commented on a change in pull request #3488:
URL: https://github.com/apache/hbase/pull/3488#discussion_r674919882



##
File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DirectStoreFSWriteStrategy.java
##
@@ -0,0 +1,258 @@
+/**
+ * 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.hadoop.hbase.regionserver;
+
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.yetus.audience.InterfaceAudience;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * HRegionFileSystemWriteStrategy implementation to be used in 
combination with
+ * PersistedStoreEngine to avoid renames when splitting and 
merging regions.
+ *
+ * To use it, define the following properties under master configuration:
+ * 1) 
+ *  hbase.hregion.file.write.strategy
+ *  
org.apache.hadoop.hbase.regionserver.DirectStoreFSWriteStrategy
+ *
+ * 2) 
+ *  hbase.hregion.merge.strategy
+ *  
org.apache.hadoop.hbase.master.assignment.DirectStoreMergeRegionsStrategy
+ *
+ *
+ * This will create the resulting merging and splitting regions directory 
straight under
+ * the table dir, instead of creating it under the temporary ".tmp" or 
".merges" dirs,
+ * as done by the default implementation.
+ */
+@InterfaceAudience.Private
+public class DirectStoreFSWriteStrategy extends HRegionFileSystemWriteStrategy 
{
+  private StoreFilePathAccessor accessor;
+  private Map>> regionSplitReferences = new 
ConcurrentHashMap<>();
+  private Map> mergeReferences = new HashMap();
+
+  public DirectStoreFSWriteStrategy(HRegionFileSystem fileSystem) throws 
IOException {
+super(fileSystem);
+this.accessor =  
StoreFileTrackingUtils.createStoreFilePathAccessor(fileSystem.conf,
+  ConnectionFactory.createConnection(fileSystem.conf));

Review comment:
   Problem is that not all write strategies would require a connection, and 
we are isolating callers from knowing which specific strategy they are calling. 




-- 
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: issues-unsubscr...@hbase.apache.org

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




[GitHub] [hbase] wchevreuil commented on a change in pull request #3488: HBASE-25393 Support split and merge region with direct insert into CF…

2021-07-22 Thread GitBox


wchevreuil commented on a change in pull request #3488:
URL: https://github.com/apache/hbase/pull/3488#discussion_r674909589



##
File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DirectStoreFSWriteStrategy.java
##
@@ -0,0 +1,258 @@
+/**
+ * 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.hadoop.hbase.regionserver;
+
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.yetus.audience.InterfaceAudience;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * HRegionFileSystemWriteStrategy implementation to be used in 
combination with
+ * PersistedStoreEngine to avoid renames when splitting and 
merging regions.
+ *
+ * To use it, define the following properties under master configuration:
+ * 1) 
+ *  hbase.hregion.file.write.strategy
+ *  
org.apache.hadoop.hbase.regionserver.DirectStoreFSWriteStrategy
+ *
+ * 2) 
+ *  hbase.hregion.merge.strategy
+ *  
org.apache.hadoop.hbase.master.assignment.DirectStoreMergeRegionsStrategy
+ *
+ *
+ * This will create the resulting merging and splitting regions directory 
straight under
+ * the table dir, instead of creating it under the temporary ".tmp" or 
".merges" dirs,
+ * as done by the default implementation.
+ */
+@InterfaceAudience.Private
+public class DirectStoreFSWriteStrategy extends HRegionFileSystemWriteStrategy 
{
+  private StoreFilePathAccessor accessor;
+  private Map>> regionSplitReferences = new 
ConcurrentHashMap<>();

Review comment:
   > Is the ConcurrentHashMap necessary? Multiple concurrent splits 
accessing this?
   
   No concurrent access, so no need for `ConcurrentHashMap`. 
   
   > Also, who cleans up the entries in this map?
   
   Yeah, this is a mistake. Entries are not being cleaned. Not a problem for 
now, as each `HRegionFileSystem` representing region being split is unique for 
the context of a single SplitProcedure, but better fix that if this is to be 
used in a different context. 




-- 
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: issues-unsubscr...@hbase.apache.org

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




[GitHub] [hbase] wchevreuil commented on a change in pull request #3488: HBASE-25393 Support split and merge region with direct insert into CF…

2021-07-22 Thread GitBox


wchevreuil commented on a change in pull request #3488:
URL: https://github.com/apache/hbase/pull/3488#discussion_r674886076



##
File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DirectStoreFSWriteStrategy.java
##
@@ -0,0 +1,258 @@
+/**
+ * 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.hadoop.hbase.regionserver;
+
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.yetus.audience.InterfaceAudience;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * HRegionFileSystemWriteStrategy implementation to be used in 
combination with
+ * PersistedStoreEngine to avoid renames when splitting and 
merging regions.

Review comment:
   I agree. We have HBASE-25396 in the roadmap, which is about sanitising 
all these pluggable components that should work together, so was not concerned 
with that in this specific jira.




-- 
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: issues-unsubscr...@hbase.apache.org

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




[GitHub] [hbase] wchevreuil commented on a change in pull request #3488: HBASE-25393 Support split and merge region with direct insert into CF…

2021-07-22 Thread GitBox


wchevreuil commented on a change in pull request #3488:
URL: https://github.com/apache/hbase/pull/3488#discussion_r674875874



##
File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MergeTableRegionsProcedure.java
##
@@ -381,6 +403,7 @@ protected void 
deserializeStateData(ProcedureStateSerializer serializer)
 for (int i = 0; i < regionsToMerge.length; i++) {
   regionsToMerge[i] = 
ProtobufUtil.toRegionInfo(mergeTableRegionsMsg.getRegionInfo(i));
 }
+createMergeStrategy(mergeTableRegionsMsg.getMergeStrategy());

Review comment:
   Yes.




-- 
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: issues-unsubscr...@hbase.apache.org

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




[GitHub] [hbase] wchevreuil commented on a change in pull request #3488: HBASE-25393 Support split and merge region with direct insert into CF…

2021-07-22 Thread GitBox


wchevreuil commented on a change in pull request #3488:
URL: https://github.com/apache/hbase/pull/3488#discussion_r674865197



##
File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MergeTableRegionsProcedure.java
##
@@ -107,6 +113,21 @@ public MergeTableRegionsProcedure(final MasterProcedureEnv 
env,
 // Preflight depends on mergedRegion being set (at least).
 preflightChecks(env, true);
 this.force = force;
+createMergeStrategy(env.getMasterConfiguration());
+  }
+
+  private void createMergeStrategy(Configuration conf) {
+String className = conf.get(MERGE_REGION_STRATEGY, 
DefaultMergeStrategy.class.getName());
+createMergeStrategy(className);
+  }
+
+  private void createMergeStrategy(String className) {

Review comment:
   Ack

##
File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MergeTableRegionsProcedure.java
##
@@ -107,6 +113,21 @@ public MergeTableRegionsProcedure(final MasterProcedureEnv 
env,
 // Preflight depends on mergedRegion being set (at least).
 preflightChecks(env, true);
 this.force = force;
+createMergeStrategy(env.getMasterConfiguration());
+  }
+
+  private void createMergeStrategy(Configuration conf) {
+String className = conf.get(MERGE_REGION_STRATEGY, 
DefaultMergeStrategy.class.getName());
+createMergeStrategy(className);
+  }
+
+  private void createMergeStrategy(String className) {
+try {
+  LOG.info("instantiating write strategy {}", className);
+  mergeStrategy = (MergeRegionsStrategy) 
Class.forName(className).newInstance();
+} catch (Exception e) {
+  LOG.error("Unable to create write strategy: {}", className, e);

Review comment:
   Ack




-- 
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: issues-unsubscr...@hbase.apache.org

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




[GitHub] [hbase] wchevreuil commented on a change in pull request #3488: HBASE-25393 Support split and merge region with direct insert into CF…

2021-07-22 Thread GitBox


wchevreuil commented on a change in pull request #3488:
URL: https://github.com/apache/hbase/pull/3488#discussion_r674855549



##
File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MergeRegionsStrategy.java
##
@@ -0,0 +1,99 @@
+/**
+ * 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.hadoop.hbase.master.assignment;
+
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.master.MasterFileSystem;
+import org.apache.hadoop.hbase.master.RegionState;
+import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
+import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
+import org.apache.yetus.audience.InterfaceAudience;
+
+import java.io.IOException;
+
+/**
+ * Region merge directory creation strategy to decouple create dir logic from
+ * MergeTableRegionsProcedure and allow for plugable 
behaviour.
+ */
+@InterfaceAudience.Private
+public abstract class MergeRegionsStrategy {
+
+  /**
+   * Creates the resulting merging region dir and files in the file system, 
then updates
+   * meta table information for the given region. Specific logic on where in 
the files system to
+   * create the region structure is delegated to 
innerMergeRegions and the
+   * actual HRegionFileSystemWriteStrategy implementation.
+   * @param env the MasterProcedureEnv wrapping several meta information 
required.
+   * @param regionsToMerge array of RegionInfo representing the regions being 
merged.
+   * @param mergedRegion the resulting merging region.
+   * @throws IOException if any error occurs while creating the region dir.
+   */
+  public void createMergedRegion(MasterProcedureEnv env, RegionInfo[] 
regionsToMerge,
+RegionInfo mergedRegion) throws IOException {
+final MasterFileSystem mfs = env.getMasterServices().getMasterFileSystem();
+final Path tabledir = CommonFSUtils.getTableDir(mfs.getRootDir(), 
regionsToMerge[0].getTable());
+final FileSystem fs = mfs.getFileSystem();
+HRegionFileSystem mergeRegionFs = innerMergeRegions(env, fs, 
regionsToMerge,
+  tabledir, mergedRegion);
+assert mergeRegionFs != null;
+mergeRegionFs.commitMergedRegion(mergedRegion);
+// Prepare to create merged regions
+env.getAssignmentManager().getRegionStates().
+  
getOrCreateRegionStateNode(mergedRegion).setState(RegionState.State.MERGING_NEW);
+  }
+
+  /**
+   * Should define specific logic about where in the file system the region 
structure should be
+   * created.
+   * @param env the MasterProcedureEnv wrapping several meta information 
required.
+   * @param fs the FileSystem instance to write the region directory.
+   * @param regionsToMerge array of RegionInfo representing the regions being 
merged.
+   * @param tableDir Path instance for the table dir.
+   * @param mergedRegion the resulting merging region.
+   * @return HRegionFileSystem for the resulting merging region.
+   * @throws IOException if any error occurs while creating the region dir.
+   */
+  abstract protected HRegionFileSystem innerMergeRegions(MasterProcedureEnv 
env, FileSystem fs,
+RegionInfo[] regionsToMerge, Path tableDir, RegionInfo mergedRegion) 
throws IOException;

Review comment:
   Ack




-- 
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: issues-unsubscr...@hbase.apache.org

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




[GitHub] [hbase] wchevreuil commented on a change in pull request #3488: HBASE-25393 Support split and merge region with direct insert into CF…

2021-07-15 Thread GitBox


wchevreuil commented on a change in pull request #3488:
URL: https://github.com/apache/hbase/pull/3488#discussion_r670584389



##
File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/DirectStoreMergeRegionsStrategy.java
##
@@ -0,0 +1,105 @@
+/**
+ * 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.hadoop.hbase.master.assignment;
+
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.TableDescriptor;
+import org.apache.hadoop.hbase.io.hfile.CacheConfig;
+import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
+import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
+import org.apache.hadoop.hbase.regionserver.HStoreFile;
+import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
+import org.apache.yetus.audience.InterfaceAudience;
+
+import java.io.IOException;
+import java.util.Collection;
+
+/**
+ * MergeRegionStrategy implementation to be used in 
combination with
+ * PersistedStoreEngine to avoid renames when merging regions.
+ *
+ * To use it, define the following properties under master configuration:

Review comment:
   Could cause confusion/inconsistencies, no? Same region splitting using 
different approaches to different stores?




-- 
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: issues-unsubscr...@hbase.apache.org

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




[GitHub] [hbase] wchevreuil commented on a change in pull request #3488: HBASE-25393 Support split and merge region with direct insert into CF…

2021-07-14 Thread GitBox


wchevreuil commented on a change in pull request #3488:
URL: https://github.com/apache/hbase/pull/3488#discussion_r669629884



##
File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/DirectStoreMergeRegionsStrategy.java
##
@@ -0,0 +1,105 @@
+/**
+ * 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.hadoop.hbase.master.assignment;
+
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.TableDescriptor;
+import org.apache.hadoop.hbase.io.hfile.CacheConfig;
+import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
+import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
+import org.apache.hadoop.hbase.regionserver.HStoreFile;
+import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
+import org.apache.yetus.audience.InterfaceAudience;
+
+import java.io.IOException;
+import java.util.Collection;
+
+/**
+ * MergeRegionStrategy implementation to be used in 
combination with
+ * PersistedStoreEngine to avoid renames when merging regions.
+ *
+ * To use it, define the following properties under master configuration:

Review comment:
   Good point. Let me modify this. I guess for consistency, maybe table 
level configuration, rather then family level.




-- 
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: issues-unsubscr...@hbase.apache.org

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