You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@gobblin.apache.org by le...@apache.org on 2021/09/10 06:20:03 UTC

[gobblin] branch master updated: [GOBBLIN-1532] Additional logging for memory-manager and partition-writer to help with writer-memory understanding

This is an automated email from the ASF dual-hosted git repository.

lesun pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/gobblin.git


The following commit(s) were added to refs/heads/master by this push:
     new 15ded96  [GOBBLIN-1532] Additional logging for memory-manager and partition-writer to help with writer-memory understanding
15ded96 is described below

commit 15ded96c99ab6219682ca3cf7e867ef6290a1b2b
Author: Lei Sun <au...@gmail.com>
AuthorDate: Thu Sep 9 23:19:57 2021 -0700

    [GOBBLIN-1532] Additional logging for memory-manager and partition-writer to help with writer-memory understanding
    
    Additional logging for memory-manager and
    partition-writer to help with writer-memory
    understanding
    
    Add extension for mapreduce record writer to emit
    stripe-size vector
    
    Remove unused imports
    
    Closes #3384 from
    autumnust/investigate_flatten_orc_meta
---
 .../orc/GobblinOrcMapreduceRecordWriter.java       | 63 ++++++++++++++++++++++
 .../mapreduce/orc/OrcKeyCompactorOutputFormat.java |  3 +-
 .../gobblin/writer/PartitionedDataWriter.java      |  2 +
 .../gobblin/writer/GobblinOrcMemoryManager.java    | 52 ++++++++++++++++++
 4 files changed, 119 insertions(+), 1 deletion(-)

diff --git a/gobblin-compaction/src/main/java/org/apache/gobblin/compaction/mapreduce/orc/GobblinOrcMapreduceRecordWriter.java b/gobblin-compaction/src/main/java/org/apache/gobblin/compaction/mapreduce/orc/GobblinOrcMapreduceRecordWriter.java
new file mode 100644
index 0000000..1ca92c5
--- /dev/null
+++ b/gobblin-compaction/src/main/java/org/apache/gobblin/compaction/mapreduce/orc/GobblinOrcMapreduceRecordWriter.java
@@ -0,0 +1,63 @@
+/*
+ * 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.gobblin.compaction.mapreduce.orc;
+
+import java.io.IOException;
+
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.orc.StripeInformation;
+import org.apache.orc.Writer;
+import org.apache.orc.mapreduce.OrcMapreduceRecordWriter;
+
+import lombok.extern.slf4j.Slf4j;
+
+import org.apache.gobblin.util.reflection.RestrictedFieldAccessingUtils;
+
+
+/**
+ * A thin extension to {@link OrcMapreduceRecordWriter} for obtaining a vector of stripe information.
+ */
+@Slf4j
+public class GobblinOrcMapreduceRecordWriter extends OrcMapreduceRecordWriter {
+  public GobblinOrcMapreduceRecordWriter(Writer writer) {
+    super(writer);
+  }
+
+  public GobblinOrcMapreduceRecordWriter(Writer writer, int rowBatchSize) {
+    super(writer, rowBatchSize);
+  }
+
+  @Override
+  public void close(TaskAttemptContext taskAttemptContext)
+      throws IOException {
+    super.close(taskAttemptContext);
+
+    // TODO: Emit this information as kafka events for ease for populating dashboard.
+    try {
+      String stripeSizeVec = ((Writer) RestrictedFieldAccessingUtils.getRestrictedFieldByReflection(
+        this, "writer", this.getClass())).getStripes()
+          .stream()
+          .mapToLong(StripeInformation::getDataLength).mapToObj(String::valueOf)
+          .reduce((x,y) -> x.concat(",").concat(y)).get();
+      log.info("The vector of Stripe-Size in enclosing writer is:" + stripeSizeVec);
+    } catch (NoSuchFieldException | IllegalAccessException e) {
+      log.error("Failed to access writer object from super class to obtain stripe information");
+    }
+  }
+}
diff --git a/gobblin-compaction/src/main/java/org/apache/gobblin/compaction/mapreduce/orc/OrcKeyCompactorOutputFormat.java b/gobblin-compaction/src/main/java/org/apache/gobblin/compaction/mapreduce/orc/OrcKeyCompactorOutputFormat.java
index c17f568..27eefcd 100644
--- a/gobblin-compaction/src/main/java/org/apache/gobblin/compaction/mapreduce/orc/OrcKeyCompactorOutputFormat.java
+++ b/gobblin-compaction/src/main/java/org/apache/gobblin/compaction/mapreduce/orc/OrcKeyCompactorOutputFormat.java
@@ -34,6 +34,7 @@ import org.apache.orc.mapreduce.OrcOutputFormat;
 import lombok.extern.slf4j.Slf4j;
 
 import org.apache.gobblin.compaction.mapreduce.CompactorOutputCommitter;
+import org.apache.gobblin.writer.GobblinOrcMemoryManager;
 import org.apache.gobblin.writer.GobblinOrcWriter;
 
 import static org.apache.gobblin.compaction.mapreduce.CompactorOutputCommitter.COMPACTION_OUTPUT_EXTENSION;
@@ -68,7 +69,7 @@ public class OrcKeyCompactorOutputFormat extends OrcOutputFormat {
 
     Path filename = getDefaultWorkFile(taskAttemptContext, extension);
     Writer writer = OrcFile.createWriter(filename,
-        org.apache.orc.mapred.OrcOutputFormat.buildOptions(conf));
+        org.apache.orc.mapred.OrcOutputFormat.buildOptions(conf).memory(new GobblinOrcMemoryManager(conf)));
     int rowBatchSize = conf.getInt(GobblinOrcWriter.ORC_WRITER_BATCH_SIZE, GobblinOrcWriter.DEFAULT_ORC_WRITER_BATCH_SIZE);
     log.info("Creating OrcMapreduceRecordWriter with row batch size = {}", rowBatchSize);
     return new OrcMapreduceRecordWriter(writer, rowBatchSize);
diff --git a/gobblin-core/src/main/java/org/apache/gobblin/writer/PartitionedDataWriter.java b/gobblin-core/src/main/java/org/apache/gobblin/writer/PartitionedDataWriter.java
index 16b9365..800676d 100644
--- a/gobblin-core/src/main/java/org/apache/gobblin/writer/PartitionedDataWriter.java
+++ b/gobblin-core/src/main/java/org/apache/gobblin/writer/PartitionedDataWriter.java
@@ -168,6 +168,8 @@ public class PartitionedDataWriter<S, D> extends WriterWrapper<D> implements Fin
                   @Override
                   public DataWriter<D> get() {
                     try {
+                      log.info(String.format("Adding one more writer to loading cache of existing writer "
+                          + "with size = %d", partitionWriters.size()));
                       return createPartitionWriter(key);
                     } catch (IOException e) {
                       throw new RuntimeException("Error creating writer", e);
diff --git a/gobblin-modules/gobblin-orc/src/main/java/org/apache/gobblin/writer/GobblinOrcMemoryManager.java b/gobblin-modules/gobblin-orc/src/main/java/org/apache/gobblin/writer/GobblinOrcMemoryManager.java
new file mode 100644
index 0000000..fe49dc8
--- /dev/null
+++ b/gobblin-modules/gobblin-orc/src/main/java/org/apache/gobblin/writer/GobblinOrcMemoryManager.java
@@ -0,0 +1,52 @@
+/*
+ * 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.gobblin.writer;
+
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.orc.impl.MemoryManagerImpl;
+
+import lombok.extern.slf4j.Slf4j;
+
+
+/**
+ * A thin layer extending {@link MemoryManagerImpl} for logging and instrumentation purpose.
+ */
+@Slf4j
+public class GobblinOrcMemoryManager extends MemoryManagerImpl {
+  public GobblinOrcMemoryManager(Configuration conf) {
+    super(conf);
+    log.info("The pool reserved for memory manager is :{}", getTotalMemoryPool());
+  }
+
+  @Override
+  public synchronized void addWriter(Path path, long requestedAllocation, Callback callback)
+      throws IOException {
+    super.addWriter(path, requestedAllocation, callback);
+    log.info("Adding writer for Path {}, Current allocation: {}", path.toString(), getAllocationScale());
+  }
+
+  @Override
+  public synchronized void removeWriter(Path path)
+      throws IOException {
+    super.removeWriter(path);
+    log.info("Closing writer for Path {}, Current allocation: {}", path.toString(), getAllocationScale());
+  }
+}