You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pinot.apache.org by GitBox <gi...@apache.org> on 2021/05/17 22:10:39 UTC

[GitHub] [incubator-pinot] sajjad-moradi commented on a change in pull request #6926: Clean up generated files for recommendation engine in case of failure

sajjad-moradi commented on a change in pull request #6926:
URL: https://github.com/apache/incubator-pinot/pull/6926#discussion_r633899902



##########
File path: pinot-controller/src/main/java/org/apache/pinot/controller/recommender/realtime/provisioning/MemoryEstimator.java
##########
@@ -127,26 +124,21 @@ public MemoryEstimator(TableConfig tableConfig, File sampleCompletedSegment, int
       _invertedIndexColumns.addAll(_tableConfig.getIndexingConfig().getInvertedIndexColumns());
     }
     _avgMultiValues = getAvgMultiValues();
-
-    _tableDataDir = new File(TMP_DIR, _tableNameWithType);
-    try {
-      FileUtils.deleteDirectory(_tableDataDir);
-    } catch (IOException e) {
-      throw new RuntimeException("Exception in deleting directory " + _tableDataDir.getAbsolutePath(), e);
-    }
-    _tableDataDir.mkdir();
+    _workingDir = workingDir;
   }
 
   /**
    * Constructor used for processing the given data characteristics (instead of completed segment)
    */
   public MemoryEstimator(TableConfig tableConfig, Schema schema, SchemaWithMetaData schemaWithMetadata,
-      int numberOfRows, int ingestionRatePerPartition, long maxUsableHostMemory, int tableRetentionHours) {
+      int numberOfRows, int ingestionRatePerPartition, long maxUsableHostMemory, int tableRetentionHours,
+      File workingDir) {

Review comment:
       Sure. 

##########
File path: pinot-controller/src/main/java/org/apache/pinot/controller/recommender/realtime/provisioning/MemoryEstimator.java
##########
@@ -519,21 +515,23 @@ private File generateData(Date now) {
         LOGGER.info("Successfully generated data file: {}", outputFile);
         return outputFile;
       } catch (Exception e) {
+        FileUtils.deleteQuietly(new File(outputDir));
         throw new RuntimeException(e);
       }
     }
 
-    private File createSegment(File csvDataFile, Date now) {
+    private File createSegment(File csvDataFile) {
 
       // create segment
       LOGGER.info("Started creating segment from file: {}", csvDataFile);
-      String outDir = getOutputDir(now, "-segment");
+      String outDir = new File(_workingDir, "segment").getAbsolutePath();
       SegmentGeneratorConfig segmentGeneratorConfig = getSegmentGeneratorConfig(csvDataFile, outDir);
       SegmentIndexCreationDriver driver = new SegmentIndexCreationDriverImpl();
       try {
         driver.init(segmentGeneratorConfig);
         driver.build();
       } catch (Exception e) {
+        FileUtils.deleteQuietly(new File(outDir));

Review comment:
       Good catch. Updated it accordingly.

##########
File path: pinot-controller/src/test/java/org/apache/pinot/controller/recommender/realtime/provisioning/MemoryEstimatorTest.java
##########
@@ -116,7 +117,7 @@ private void runTest(String schemaFileName, Consumer<String> assertFunc) throws
     assertFunc.accept(metadata);
 
     // cleanup
-    FileUtils.deleteDirectory(generatedSegment);
+    FileUtils.deleteDirectory(workingDir);

Review comment:
       Please refer to the next comment.

##########
File path: pinot-tools/src/main/java/org/apache/pinot/tools/admin/command/RealtimeProvisioningHelperCommand.java
##########
@@ -278,6 +280,7 @@ public boolean execute()
     displayResults(memoryEstimator.getConsumingMemoryPerHost(), numHosts, numHours);
     LOGGER.info("\nTotal number of segments queried per host (for all partitions)");
     displayResults(memoryEstimator.getNumSegmentsQueriedPerHost(), numHosts, numHours);
+    memoryEstimator.cleanup();

Review comment:
       Good point. I agree.
   Note that there are two ways segments are generated: 
   1. Directly from SegmentGenerator.generate()
   2. Implicitly in MemoryEstimator.estimateMemory()
   For case 2, as you mentioned, we should clean it automatically and not delegate that to the caller of estimateMemory.
   For case 1, it's up to the caller if they want to keep or delete the generated segment. So SegmentGenerator doesn't delete the segment automatically.




-- 
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.

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



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org