[ 
https://issues.apache.org/jira/browse/BEAM-7545?focusedWorklogId=268025&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-268025
 ]

ASF GitHub Bot logged work on BEAM-7545:
----------------------------------------

                Author: ASF GitHub Bot
            Created on: 26/Jun/19 22:00
            Start Date: 26/Jun/19 22:00
    Worklog Time Spent: 10m 
      Work Description: akedin commented on pull request #8951: [BEAM-7545] 
Adding RowCount to TextTable
URL: https://github.com/apache/beam/pull/8951#discussion_r297877972
 
 

 ##########
 File path: 
sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextRowCountEstimator.java
 ##########
 @@ -0,0 +1,154 @@
+/*
+ * 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.beam.sdk.io;
+
+import com.google.auto.value.AutoValue;
+import java.io.IOException;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.io.fs.EmptyMatchTreatment;
+import org.apache.beam.sdk.io.fs.MatchResult;
+import org.apache.beam.sdk.io.range.OffsetRange;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.options.ValueProvider;
+
+/** This returns a row count estimation for files associated with a file 
pattern. */
+@AutoValue
+public abstract class TextRowCountEstimator {
+  private static final long DEFAULT_NUM_BYTES_LINES = 1024L;
+  private static final Compression DEFAULT_COMPRESSION = Compression.AUTO;
+  private static final FileIO.ReadMatches.DirectoryTreatment 
DEFAULT_DIRECTORY_TREATMENT =
+      FileIO.ReadMatches.DirectoryTreatment.SKIP;
+  private static final EmptyMatchTreatment DEFAULT_EMPTY_MATCH_TREATMENT =
+      EmptyMatchTreatment.DISALLOW;
+
+  public abstract int getNumSampledFiles();
+
+  public boolean isSamplingAllFiles() {
+    return getNumSampledFiles() == -1;
+  }
+
+  public abstract long getNumSampledBytes();
+
+  @Nullable
+  @SuppressWarnings("mutable")
+  public abstract byte[] getDelimiters();
+
+  public abstract String getFilePattern();
+
+  public abstract Compression getCompression();
+
+  public abstract EmptyMatchTreatment getEmptyMatchTreatment();
+
+  public abstract FileIO.ReadMatches.DirectoryTreatment 
getDirectoryTreatment();
+
+  public static TextRowCountEstimator.Builder builder() {
+    return (new AutoValue_TextRowCountEstimator.Builder())
+        .sampleAllFiles()
+        .setNumSampledBytes(DEFAULT_NUM_BYTES_LINES)
+        .setCompression(DEFAULT_COMPRESSION)
+        .setDirectoryTreatment(DEFAULT_DIRECTORY_TREATMENT)
+        .setEmptyMatchTreatment(DEFAULT_EMPTY_MATCH_TREATMENT);
+  }
+
+  /** Builder for {@link org.apache.beam.sdk.io.TextRowCountEstimator}. */
+  @AutoValue.Builder
+  public abstract static class Builder {
+
+    public abstract Builder setNumSampledFiles(int numSampledFiles);
+
+    public abstract Builder setNumSampledBytes(long numSampledBytes);
+
+    public abstract Builder setDirectoryTreatment(
+        FileIO.ReadMatches.DirectoryTreatment directoryTreatment);
+
+    public abstract Builder setCompression(Compression compression);
+
+    public abstract Builder setDelimiters(byte[] delimiters);
+
+    public abstract Builder setFilePattern(String filePattern);
+
+    public abstract Builder setEmptyMatchTreatment(EmptyMatchTreatment 
emptyMatchTreatment);
+
+    public Builder sampleAllFiles() {
 
 Review comment:
   I think it's simpler to make this a separate abstract boolean method, so 
that you don't have to actually implement this and the getter
 
----------------------------------------------------------------
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:
[email protected]


Issue Time Tracking
-------------------

    Worklog Id:     (was: 268025)
    Time Spent: 2h 10m  (was: 2h)

> Row Count Estimation for CSV TextTable
> --------------------------------------
>
>                 Key: BEAM-7545
>                 URL: https://issues.apache.org/jira/browse/BEAM-7545
>             Project: Beam
>          Issue Type: New Feature
>          Components: dsl-sql
>            Reporter: Alireza Samadianzakaria
>            Assignee: Alireza Samadianzakaria
>            Priority: Major
>          Time Spent: 2h 10m
>  Remaining Estimate: 0h
>
> Implementing Row Count Estimation for CSV Tables by reading the first few 
> lines of the file and estimating the number of records based on the length of 
> these lines and the total length of the file.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to