[GitHub] rdblue commented on a change in pull request #7: Allow custom hadoop properties to be loaded in the Spark data source
rdblue commented on a change in pull request #7: Allow custom hadoop properties to be loaded in the Spark data source URL: https://github.com/apache/incubator-iceberg/pull/7#discussion_r240717374 ## File path: spark/src/main/java/com/netflix/iceberg/spark/source/IcebergSource.java ## @@ -89,30 +92,51 @@ public DataSourceReader createReader(DataSourceOptions options) { .toUpperCase(Locale.ENGLISH)); } -return Optional.of(new Writer(table, lazyConf(), format)); +return Optional.of(new Writer(table, conf, format)); } - protected Table findTable(DataSourceOptions options) { + protected Table findTable(DataSourceOptions options, Configuration conf) { Optional location = options.get("path"); Preconditions.checkArgument(location.isPresent(), "Cannot open table without a location: path is not set"); -HadoopTables tables = new HadoopTables(lazyConf()); +HadoopTables tables = new HadoopTables(conf); return tables.load(location.get()); } - protected SparkSession lazySparkSession() { + private SparkSession lazySparkSession() { if (lazySpark == null) { this.lazySpark = SparkSession.builder().getOrCreate(); } return lazySpark; } - protected Configuration lazyConf() { + private Configuration lazyBaseConf() { if (lazyConf == null) { this.lazyConf = lazySparkSession().sparkContext().hadoopConfiguration(); } return lazyConf; } + + private Table getTableAndResolveHadoopConfiguration( + DataSourceOptions options, Configuration conf) { +// Overwrite configurations from the Spark Context with configurations from the options. +mergeIcebergHadoopConfs(conf, options.asMap(), true); +Table table = findTable(options, conf); +// Set confs from table properties, but do not overwrite options from the Spark Context with +// configurations from the table +mergeIcebergHadoopConfs(conf, table.properties(), false); Review comment: I think this still needs to be true, in which case we can remove the option. Table properties still need to override those set in the Hadoop Configuration. Then we re-apply the ones from options to fix up precedence. This is an automated message from the Apache Git Service. To respond to the message, please log on GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services
[GitHub] rdblue commented on a change in pull request #7: Allow custom hadoop properties to be loaded in the Spark data source
rdblue commented on a change in pull request #7: Allow custom hadoop properties to be loaded in the Spark data source URL: https://github.com/apache/incubator-iceberg/pull/7#discussion_r240717846 ## File path: spark/src/main/java/com/netflix/iceberg/spark/source/IcebergSource.java ## @@ -89,30 +92,51 @@ public DataSourceReader createReader(DataSourceOptions options) { .toUpperCase(Locale.ENGLISH)); } -return Optional.of(new Writer(table, lazyConf(), format)); +return Optional.of(new Writer(table, conf, format)); } - protected Table findTable(DataSourceOptions options) { + protected Table findTable(DataSourceOptions options, Configuration conf) { Optional location = options.get("path"); Preconditions.checkArgument(location.isPresent(), "Cannot open table without a location: path is not set"); -HadoopTables tables = new HadoopTables(lazyConf()); +HadoopTables tables = new HadoopTables(conf); return tables.load(location.get()); } - protected SparkSession lazySparkSession() { + private SparkSession lazySparkSession() { if (lazySpark == null) { this.lazySpark = SparkSession.builder().getOrCreate(); } return lazySpark; } - protected Configuration lazyConf() { + private Configuration lazyBaseConf() { if (lazyConf == null) { this.lazyConf = lazySparkSession().sparkContext().hadoopConfiguration(); } return lazyConf; } + + private Table getTableAndResolveHadoopConfiguration( + DataSourceOptions options, Configuration conf) { +// Overwrite configurations from the Spark Context with configurations from the options. +mergeIcebergHadoopConfs(conf, options.asMap(), true); +Table table = findTable(options, conf); +// Set confs from table properties, but do not overwrite options from the Spark Context with +// configurations from the table +mergeIcebergHadoopConfs(conf, table.properties(), false); +// Re-overwrite values set in options and table properties but were not in the environment. +mergeIcebergHadoopConfs(conf, options.asMap(), true); +return table; + } + + private static void mergeIcebergHadoopConfs( + Configuration baseConf, Map options, boolean overwrite) { +options.keySet().stream() +.filter(key -> key.startsWith("iceberg.hadoop")) +.filter(key -> overwrite || baseConf.get(key.replaceFirst("iceberg.hadoop", "")) == null) Review comment: Doesn't overwrite discard all keys? I don't think it matters now because it isn't needed anymore. This is an automated message from the Apache Git Service. To respond to the message, please log on GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services
[GitHub] mccheah commented on a change in pull request #7: Allow custom hadoop properties to be loaded in the Spark data source
mccheah commented on a change in pull request #7: Allow custom hadoop properties to be loaded in the Spark data source URL: https://github.com/apache/incubator-iceberg/pull/7#discussion_r240760798 ## File path: spark/src/main/java/com/netflix/iceberg/spark/source/IcebergSource.java ## @@ -89,30 +92,51 @@ public DataSourceReader createReader(DataSourceOptions options) { .toUpperCase(Locale.ENGLISH)); } -return Optional.of(new Writer(table, lazyConf(), format)); +return Optional.of(new Writer(table, conf, format)); } - protected Table findTable(DataSourceOptions options) { + protected Table findTable(DataSourceOptions options, Configuration conf) { Optional location = options.get("path"); Preconditions.checkArgument(location.isPresent(), "Cannot open table without a location: path is not set"); -HadoopTables tables = new HadoopTables(lazyConf()); +HadoopTables tables = new HadoopTables(conf); return tables.load(location.get()); } - protected SparkSession lazySparkSession() { + private SparkSession lazySparkSession() { if (lazySpark == null) { this.lazySpark = SparkSession.builder().getOrCreate(); } return lazySpark; } - protected Configuration lazyConf() { + private Configuration lazyBaseConf() { if (lazyConf == null) { this.lazyConf = lazySparkSession().sparkContext().hadoopConfiguration(); } return lazyConf; } + + private Table getTableAndResolveHadoopConfiguration( + DataSourceOptions options, Configuration conf) { +// Overwrite configurations from the Spark Context with configurations from the options. +mergeIcebergHadoopConfs(conf, options.asMap(), true); +Table table = findTable(options, conf); +// Set confs from table properties, but do not overwrite options from the Spark Context with +// configurations from the table +mergeIcebergHadoopConfs(conf, table.properties(), false); Review comment: Hm, I would think that properties set in the JVM, particularly if set on the Spark Context via `spark.hadoop.*`, should take precedence over the table properties. This is an automated message from the Apache Git Service. To respond to the message, please log on GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services
[GitHub] mccheah commented on issue #45: Lazily submit tasks in ParallelIterable and add cancellation.
mccheah commented on issue #45: Lazily submit tasks in ParallelIterable and add cancellation. URL: https://github.com/apache/incubator-iceberg/pull/45#issuecomment-446334195 Still looking over this, but separately I wonder if we've looked into Java's built in parallel iterators, or [spliterators](https://docs.oracle.com/javase/8/docs/api/java/util/Spliterator.html). This is an automated message from the Apache Git Service. To respond to the message, please log on GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services
[GitHub] rdblue commented on a change in pull request #7: Allow custom hadoop properties to be loaded in the Spark data source
rdblue commented on a change in pull request #7: Allow custom hadoop properties to be loaded in the Spark data source URL: https://github.com/apache/incubator-iceberg/pull/7#discussion_r240763430 ## File path: spark/src/main/java/com/netflix/iceberg/spark/source/IcebergSource.java ## @@ -89,30 +92,51 @@ public DataSourceReader createReader(DataSourceOptions options) { .toUpperCase(Locale.ENGLISH)); } -return Optional.of(new Writer(table, lazyConf(), format)); +return Optional.of(new Writer(table, conf, format)); } - protected Table findTable(DataSourceOptions options) { + protected Table findTable(DataSourceOptions options, Configuration conf) { Optional location = options.get("path"); Preconditions.checkArgument(location.isPresent(), "Cannot open table without a location: path is not set"); -HadoopTables tables = new HadoopTables(lazyConf()); +HadoopTables tables = new HadoopTables(conf); return tables.load(location.get()); } - protected SparkSession lazySparkSession() { + private SparkSession lazySparkSession() { if (lazySpark == null) { this.lazySpark = SparkSession.builder().getOrCreate(); } return lazySpark; } - protected Configuration lazyConf() { + private Configuration lazyBaseConf() { if (lazyConf == null) { this.lazyConf = lazySparkSession().sparkContext().hadoopConfiguration(); } return lazyConf; } + + private Table getTableAndResolveHadoopConfiguration( + DataSourceOptions options, Configuration conf) { +// Overwrite configurations from the Spark Context with configurations from the options. +mergeIcebergHadoopConfs(conf, options.asMap(), true); +Table table = findTable(options, conf); +// Set confs from table properties, but do not overwrite options from the Spark Context with +// configurations from the table +mergeIcebergHadoopConfs(conf, table.properties(), false); Review comment: Values set in the Configuration are session specific and what we want is to move to table settings instead of Spark settings for configuration like Parquet row group size that are tied to the data. Write-specific settings from the write config can override. Table settings should take priority over session-wide settings because session-wide config would apply for all tables, and that's not usually appropriate like the row group size example. This is an automated message from the Apache Git Service. To respond to the message, please log on GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services
[GitHub] rdblue commented on issue #45: Lazily submit tasks in ParallelIterable and add cancellation.
rdblue commented on issue #45: Lazily submit tasks in ParallelIterable and add cancellation. URL: https://github.com/apache/incubator-iceberg/pull/45#issuecomment-446338379 I'm not that impressed with the Spliterator API, and it wouldn't really help here where we want to control when background tasks are submitted and how they run. We could supply some implementation of tasks that is spliterable and use that as a fixed number of tasks, but that doesn't allow us to balance tasks across threads. It also doesn't allow us to avoid submitting tasks to read manifest contents into memory, which is the purpose of this commit. This is an automated message from the Apache Git Service. To respond to the message, please log on GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services
[GitHub] mccheah commented on a change in pull request #45: Lazily submit tasks in ParallelIterable and add cancellation.
mccheah commented on a change in pull request #45: Lazily submit tasks in ParallelIterable and add cancellation. URL: https://github.com/apache/incubator-iceberg/pull/45#discussion_r240776853 ## File path: core/src/main/java/com/netflix/iceberg/util/ParallelIterable.java ## @@ -19,73 +19,124 @@ package com.netflix.iceberg.util; +import com.google.common.base.Preconditions; +import com.google.common.collect.Iterables; +import com.netflix.iceberg.io.CloseableGroup; +import java.io.Closeable; import java.util.Iterator; import java.util.NoSuchElementException; import java.util.concurrent.ConcurrentLinkedQueue; -import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; import java.util.concurrent.Future; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.TimeoutException; -public class ParallelIterable implements Iterable { +public class ParallelIterable extends CloseableGroup implements Iterable { private final Iterable> iterables; - private final ExecutorService trackingPool; private final ExecutorService workerPool; public ParallelIterable(Iterable> iterables, - ExecutorService trackingPool, ExecutorService workerPool) { this.iterables = iterables; -this.trackingPool = trackingPool; this.workerPool = workerPool; } @Override public Iterator iterator() { -return new ParallelIterator<>(iterables, trackingPool, workerPool); +ParallelIterator iter = new ParallelIterator<>(iterables, workerPool); +addCloseable(iter); +return iter; } - private static class ParallelIterator implements Iterator { + private static class ParallelIterator implements Iterator, Closeable { Review comment: I'm wondering if there's a more idiomatic way to do this, particularly one that doesn't require both: 1. Busy waiting. Generally it's a flag in concurrent programming if busy waiting is used over alternative primitives like locks, conditions, queues, monitors, etc. 2. Manual tracking of tasks by index. I came up with the following alternative. Apologize that it has to be in pseudo-code form; due to the nature of the problem it's pretty hard to explain without the code. Let's see how this works out: ``` class ParallelIterator implements Iterator, Closeable { private LinkedList availableValues; private LinkedList>> runningTasks; private ExecutorService threadPool; private Iterator> pendingValues; // Constructor etc. boolean hasNext() { return !runningTasks.isEmpty() || !availableValues.isEmpty() || !pendingValues.isEmpty(); } T next() { if (!availableValues.isEmpty()) { return availableValues.poll(); } if (!runningTasks.isEmpty()) { availableValues.addAll(runningTasks.poll().get()); return next(); // Or availableValues.poll() if we don't like recursion } if (pendingValues.hasNext()) { // Buffer / eagerly submit some set of tasks, i.e. lookahead. for (int i = 0; i < TASK_COUNT && pendingValues.hasNext(); i++) { Iterable nextPendingValues = pendingValues.next(); Future> nextRunningTask = threadPool.submit(() -> ImmutableList.copyOf(nextPendingValues)); runningTasks.add(nextRunningTask); } return next(); // Recursive call that checks will now check based on running tasks } throw error; // No values remaining } } ``` The general idea is to keep a running iterator over the backing iterable. When calling `next()`, submit tasks that are buffered into a worker queue of futures; each future represents computing the next group of values. Then on `next()`: - Get an available value from a completed task, if possible - Else check the work queue and see if a new batch of values is ready - Otherwise submit more work and wait What do you think about this approach? The advantages are: - No busy waiting - No need to maintain indices manually. Everything is done via collection primitives (`poll`, `iterator`, etc.) There's a few ways this framework can be adjusted. For example on `next`, if we determine that there is only some minimum number of running tasks remaining, we can choose to eagerly submit work ahead of the user actually requesting for those values - thereby we pipeline the main thread's work on the values with the worker thread's work that produces the values. This is an automated message from the Apache Git Service. To respond to the message, please log on GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services
[GitHub] mccheah commented on a change in pull request #7: Allow custom hadoop properties to be loaded in the Spark data source
mccheah commented on a change in pull request #7: Allow custom hadoop properties to be loaded in the Spark data source URL: https://github.com/apache/incubator-iceberg/pull/7#discussion_r240778558 ## File path: spark/src/main/java/com/netflix/iceberg/spark/source/IcebergSource.java ## @@ -89,30 +92,51 @@ public DataSourceReader createReader(DataSourceOptions options) { .toUpperCase(Locale.ENGLISH)); } -return Optional.of(new Writer(table, lazyConf(), format)); +return Optional.of(new Writer(table, conf, format)); } - protected Table findTable(DataSourceOptions options) { + protected Table findTable(DataSourceOptions options, Configuration conf) { Optional location = options.get("path"); Preconditions.checkArgument(location.isPresent(), "Cannot open table without a location: path is not set"); -HadoopTables tables = new HadoopTables(lazyConf()); +HadoopTables tables = new HadoopTables(conf); return tables.load(location.get()); } - protected SparkSession lazySparkSession() { + private SparkSession lazySparkSession() { if (lazySpark == null) { this.lazySpark = SparkSession.builder().getOrCreate(); } return lazySpark; } - protected Configuration lazyConf() { + private Configuration lazyBaseConf() { if (lazyConf == null) { this.lazyConf = lazySparkSession().sparkContext().hadoopConfiguration(); } return lazyConf; } + + private Table getTableAndResolveHadoopConfiguration( + DataSourceOptions options, Configuration conf) { +// Overwrite configurations from the Spark Context with configurations from the options. +mergeIcebergHadoopConfs(conf, options.asMap(), true); +Table table = findTable(options, conf); +// Set confs from table properties, but do not overwrite options from the Spark Context with +// configurations from the table +mergeIcebergHadoopConfs(conf, table.properties(), false); Review comment: That's fair enough, I suppose as long as the behavior is well documented it should be clear to the user on how to get the final configurations they want. This is an automated message from the Apache Git Service. To respond to the message, please log on GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services
[GitHub] rdblue commented on issue #6: Support customizing the location where data is written in Spark
rdblue commented on issue #6: Support customizing the location where data is written in Spark URL: https://github.com/apache/incubator-iceberg/pull/6#issuecomment-446352792 Merged. Thanks @mccheah! This is an automated message from the Apache Git Service. To respond to the message, please log on GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services
[GitHub] rdblue closed pull request #6: Support customizing the location where data is written in Spark
rdblue closed pull request #6: Support customizing the location where data is written in Spark URL: https://github.com/apache/incubator-iceberg/pull/6 This is a PR merged from a forked repository. As GitHub hides the original diff on merge, it is displayed below for the sake of provenance: As this is a foreign pull request (from a fork), the diff is supplied below (as it won't show otherwise due to GitHub magic): diff --git a/core/src/main/java/com/netflix/iceberg/TableProperties.java b/core/src/main/java/com/netflix/iceberg/TableProperties.java index e522f84..0d99c7e 100644 --- a/core/src/main/java/com/netflix/iceberg/TableProperties.java +++ b/core/src/main/java/com/netflix/iceberg/TableProperties.java @@ -67,6 +67,11 @@ public static final String OBJECT_STORE_PATH = "write.object-storage.path"; + // This only applies to files written after this property is set. Files previously written aren't relocated to + // reflect this parameter. + // If not set, defaults to a "data" folder underneath the root path of the table. + public static final String WRITE_NEW_DATA_LOCATION = "write.folder-storage.path"; + public static final String MANIFEST_LISTS_ENABLED = "write.manifest-lists.enabled"; public static final boolean MANIFEST_LISTS_ENABLED_DEFAULT = false; } diff --git a/spark/src/main/java/com/netflix/iceberg/spark/source/Writer.java b/spark/src/main/java/com/netflix/iceberg/spark/source/Writer.java index bed2cf6..c9d3a7b 100644 --- a/spark/src/main/java/com/netflix/iceberg/spark/source/Writer.java +++ b/spark/src/main/java/com/netflix/iceberg/spark/source/Writer.java @@ -32,6 +32,7 @@ import com.netflix.iceberg.PartitionSpec; import com.netflix.iceberg.Schema; import com.netflix.iceberg.Table; +import com.netflix.iceberg.TableProperties; import com.netflix.iceberg.avro.Avro; import com.netflix.iceberg.exceptions.RuntimeIOException; import com.netflix.iceberg.hadoop.HadoopInputFile; @@ -164,7 +165,9 @@ private int propertyAsInt(String property, int defaultValue) { } private String dataLocation() { -return new Path(new Path(table.location()), "data").toString(); +return table.properties().getOrDefault( +TableProperties.WRITE_NEW_DATA_LOCATION, +new Path(new Path(table.location()), "data").toString()); } @Override diff --git a/spark/src/test/java/com/netflix/iceberg/spark/data/AvroDataTest.java b/spark/src/test/java/com/netflix/iceberg/spark/data/AvroDataTest.java index f84c6fe..bc74908 100644 --- a/spark/src/test/java/com/netflix/iceberg/spark/data/AvroDataTest.java +++ b/spark/src/test/java/com/netflix/iceberg/spark/data/AvroDataTest.java @@ -38,7 +38,7 @@ protected abstract void writeAndValidate(Schema schema) throws IOException; - private static final StructType SUPPORTED_PRIMITIVES = StructType.of( + protected static final StructType SUPPORTED_PRIMITIVES = StructType.of( required(100, "id", LongType.get()), optional(101, "data", Types.StringType.get()), required(102, "b", Types.BooleanType.get()), diff --git a/spark/src/test/java/com/netflix/iceberg/spark/source/TestDataFrameWrites.java b/spark/src/test/java/com/netflix/iceberg/spark/source/TestDataFrameWrites.java index 3b0d32b..05f8f80 100644 --- a/spark/src/test/java/com/netflix/iceberg/spark/source/TestDataFrameWrites.java +++ b/spark/src/test/java/com/netflix/iceberg/spark/source/TestDataFrameWrites.java @@ -32,10 +32,12 @@ import com.netflix.iceberg.spark.data.AvroDataTest; import com.netflix.iceberg.spark.data.RandomData; import com.netflix.iceberg.spark.data.SparkAvroReader; +import com.netflix.iceberg.types.Types; import org.apache.avro.generic.GenericData.Record; import org.apache.hadoop.conf.Configuration; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.sql.DataFrameWriter; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.SparkSession; @@ -43,10 +45,12 @@ import org.junit.AfterClass; import org.junit.Assert; import org.junit.BeforeClass; +import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; import java.io.File; import java.io.IOException; +import java.net.URI; import java.util.List; import static com.netflix.iceberg.spark.SparkSchemaUtil.convert; @@ -57,7 +61,7 @@ public class TestDataFrameWrites extends AvroDataTest { private static final Configuration CONF = new Configuration(); - private String format = null; + private final String format; @Parameterized.Parameters public static Object[][] parameters() { @@ -90,23 +94,43 @@ public static void stopSpark() { @Override protected void writeAndValidate(Schema schema) throws IOException { +File location = createTableFolder(); +Table table = createTable(schema, location); +writeAndValidateWithLocations(table, location, new File(location, "data")); + } + + @Test + p
[GitHub] mccheah commented on issue #20: Encryption in Data Files
mccheah commented on issue #20: Encryption in Data Files URL: https://github.com/apache/incubator-iceberg/issues/20#issuecomment-446404505 @rdblue @omalley I moved the architecture to https://docs.google.com/document/d/1LptmFB7az2rLnou27QK_KKHgjcA5vKza0dWj4h8fkno/edit?usp=sharing. Please take a look and comment, and then we can continue with implementation. This is an automated message from the Apache Git Service. To respond to the message, please log on GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services
[GitHub] rdsr commented on a change in pull request #45: Lazily submit tasks in ParallelIterable and add cancellation.
rdsr commented on a change in pull request #45: Lazily submit tasks in ParallelIterable and add cancellation. URL: https://github.com/apache/incubator-iceberg/pull/45#discussion_r240885973 ## File path: core/src/main/java/com/netflix/iceberg/util/ParallelIterable.java ## @@ -19,73 +19,124 @@ package com.netflix.iceberg.util; +import com.google.common.base.Preconditions; +import com.google.common.collect.Iterables; +import com.netflix.iceberg.io.CloseableGroup; +import java.io.Closeable; import java.util.Iterator; import java.util.NoSuchElementException; import java.util.concurrent.ConcurrentLinkedQueue; -import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; import java.util.concurrent.Future; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.TimeoutException; -public class ParallelIterable implements Iterable { +public class ParallelIterable extends CloseableGroup implements Iterable { private final Iterable> iterables; - private final ExecutorService trackingPool; private final ExecutorService workerPool; public ParallelIterable(Iterable> iterables, - ExecutorService trackingPool, ExecutorService workerPool) { this.iterables = iterables; -this.trackingPool = trackingPool; this.workerPool = workerPool; } @Override public Iterator iterator() { -return new ParallelIterator<>(iterables, trackingPool, workerPool); +ParallelIterator iter = new ParallelIterator<>(iterables, workerPool); +addCloseable(iter); +return iter; } - private static class ParallelIterator implements Iterator { + private static class ParallelIterator implements Iterator, Closeable { +private final Iterator tasks; +private final ExecutorService workerPool; +private final Future[] taskFutures; private final ConcurrentLinkedQueue queue = new ConcurrentLinkedQueue<>(); -private final Future taskFuture; - -public ParallelIterator(Iterable> iterables, -ExecutorService trackingPool, -ExecutorService workerPool) { - this.taskFuture = trackingPool.submit(() -> { -Tasks.foreach(iterables) -.noRetry().stopOnFailure().throwFailureWhenFinished() -.executeWith(workerPool) -.run(iterable -> { - for (T item : iterable) { -queue.add(item); - } -}); -return true; - }); +private boolean closed = false; + +private ParallelIterator(Iterable> iterables, + ExecutorService workerPool) { + this.tasks = Iterables.transform(iterables, iterable -> + (Runnable) () -> { +for (T item : iterable) { + queue.add(item); +} + }).iterator(); + this.workerPool = workerPool; + // submit 2 tasks per worker at a time + this.taskFutures = new Future[2 * ThreadPools.WORKER_THREAD_POOL_SIZE]; +} + +@Override +public void close() { + // cancel background tasks + for (int i = 0; i < taskFutures.length; i += 1) { +if (taskFutures[i] != null && !taskFutures[i].isDone()) { + taskFutures[i].cancel(true); +} + } + this.closed = true; +} + +/** + * Checks on running tasks and submits new tasks if needed. + * + * This should not be called after {@link #close()}. + * + * @return true if there are pending tasks, false otherwise + */ +private boolean checkTasks() { + boolean hasRunningTask = false; + + for (int i = 0; i < taskFutures.length; i += 1) { +if (taskFutures[i] == null || taskFutures[i].isDone()) { + taskFutures[i] = submitNextTask(); +} + +if (taskFutures[i] != null) { + hasRunningTask = true; +} + } + + return tasks.hasNext() || hasRunningTask; +} + +private Future submitNextTask() { + if (tasks.hasNext()) { +return workerPool.submit(tasks.next()); + } + return null; } @Override public synchronized boolean hasNext() { + Preconditions.checkState(!closed, "Already closed"); + + // if the consumer is processing records more slowly than the producers, then this check will + // prevent tasks from being submitted. while the producers are running, this will always + // return here before running checkTasks. when enough of the tasks are finished that the + // consumer catches up, then lots of new tasks will be submitted at once. this behavior is + // okay because it ensures that records are not stacking up waiting to be consumed and taking + // up memory. + // + // consumers that process results quickly will periodically exhaust the queue and submit new + // tasks when checkTasks runs. fast consumers should not be delaye
[GitHub] rdsr commented on a change in pull request #45: Lazily submit tasks in ParallelIterable and add cancellation.
rdsr commented on a change in pull request #45: Lazily submit tasks in ParallelIterable and add cancellation. URL: https://github.com/apache/incubator-iceberg/pull/45#discussion_r240886083 ## File path: core/src/main/java/com/netflix/iceberg/util/ThreadPools.java ## @@ -27,42 +27,21 @@ import java.util.concurrent.ThreadPoolExecutor; public class ThreadPools { - public static final String PLANNER_THREAD_POOL_SIZE_PROP = - SystemProperties.PLANNER_THREAD_POOL_SIZE_PROP; Review comment: Seems like this property is no longer required. Would it make sense to remove it? This is an automated message from the Apache Git Service. To respond to the message, please log on GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services
[GitHub] rdsr commented on a change in pull request #45: Lazily submit tasks in ParallelIterable and add cancellation.
rdsr commented on a change in pull request #45: Lazily submit tasks in ParallelIterable and add cancellation. URL: https://github.com/apache/incubator-iceberg/pull/45#discussion_r240886393 ## File path: core/src/main/java/com/netflix/iceberg/util/ThreadPools.java ## @@ -27,42 +27,21 @@ import java.util.concurrent.ThreadPoolExecutor; public class ThreadPools { - public static final String PLANNER_THREAD_POOL_SIZE_PROP = - SystemProperties.PLANNER_THREAD_POOL_SIZE_PROP; public static final String WORKER_THREAD_POOL_SIZE_PROP = SystemProperties.WORKER_THREAD_POOL_SIZE_PROP; - private static ExecutorService PLANNER_POOL = MoreExecutors.getExitingExecutorService( - (ThreadPoolExecutor) Executors.newFixedThreadPool( - getPoolSize(PLANNER_THREAD_POOL_SIZE_PROP, 4), - new ThreadFactoryBuilder() - .setDaemon(true) - .setNameFormat("iceberg-planner-pool-%d") - .build())); + public static final int WORKER_THREAD_POOL_SIZE = getPoolSize( + WORKER_THREAD_POOL_SIZE_PROP, + Runtime.getRuntime().availableProcessors()); private static ExecutorService WORKER_POOL = MoreExecutors.getExitingExecutorService( (ThreadPoolExecutor) Executors.newFixedThreadPool( - getPoolSize(WORKER_THREAD_POOL_SIZE_PROP, Runtime.getRuntime().availableProcessors()), + WORKER_THREAD_POOL_SIZE, new ThreadFactoryBuilder() .setDaemon(true) Review comment: I think `MoreExecutors.getExitingExecutorService ` already creates daemon threads This is an automated message from the Apache Git Service. To respond to the message, please log on GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services
[GitHub] rdsr commented on a change in pull request #45: Lazily submit tasks in ParallelIterable and add cancellation.
rdsr commented on a change in pull request #45: Lazily submit tasks in ParallelIterable and add cancellation. URL: https://github.com/apache/incubator-iceberg/pull/45#discussion_r240886083 ## File path: core/src/main/java/com/netflix/iceberg/util/ThreadPools.java ## @@ -27,42 +27,21 @@ import java.util.concurrent.ThreadPoolExecutor; public class ThreadPools { - public static final String PLANNER_THREAD_POOL_SIZE_PROP = - SystemProperties.PLANNER_THREAD_POOL_SIZE_PROP; Review comment: nit: Seems like this property is no longer required. Would it make sense to remove it? This is an automated message from the Apache Git Service. To respond to the message, please log on GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services
[GitHub] rdsr commented on a change in pull request #45: Lazily submit tasks in ParallelIterable and add cancellation.
rdsr commented on a change in pull request #45: Lazily submit tasks in ParallelIterable and add cancellation. URL: https://github.com/apache/incubator-iceberg/pull/45#discussion_r240886393 ## File path: core/src/main/java/com/netflix/iceberg/util/ThreadPools.java ## @@ -27,42 +27,21 @@ import java.util.concurrent.ThreadPoolExecutor; public class ThreadPools { - public static final String PLANNER_THREAD_POOL_SIZE_PROP = - SystemProperties.PLANNER_THREAD_POOL_SIZE_PROP; public static final String WORKER_THREAD_POOL_SIZE_PROP = SystemProperties.WORKER_THREAD_POOL_SIZE_PROP; - private static ExecutorService PLANNER_POOL = MoreExecutors.getExitingExecutorService( - (ThreadPoolExecutor) Executors.newFixedThreadPool( - getPoolSize(PLANNER_THREAD_POOL_SIZE_PROP, 4), - new ThreadFactoryBuilder() - .setDaemon(true) - .setNameFormat("iceberg-planner-pool-%d") - .build())); + public static final int WORKER_THREAD_POOL_SIZE = getPoolSize( + WORKER_THREAD_POOL_SIZE_PROP, + Runtime.getRuntime().availableProcessors()); private static ExecutorService WORKER_POOL = MoreExecutors.getExitingExecutorService( (ThreadPoolExecutor) Executors.newFixedThreadPool( - getPoolSize(WORKER_THREAD_POOL_SIZE_PROP, Runtime.getRuntime().availableProcessors()), + WORKER_THREAD_POOL_SIZE, new ThreadFactoryBuilder() .setDaemon(true) Review comment: nit: I think `MoreExecutors.getExitingExecutorService ` already creates daemon threads This is an automated message from the Apache Git Service. To respond to the message, please log on GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services
[GitHub] rdsr commented on a change in pull request #45: Lazily submit tasks in ParallelIterable and add cancellation.
rdsr commented on a change in pull request #45: Lazily submit tasks in ParallelIterable and add cancellation. URL: https://github.com/apache/incubator-iceberg/pull/45#discussion_r240885973 ## File path: core/src/main/java/com/netflix/iceberg/util/ParallelIterable.java ## @@ -19,73 +19,124 @@ package com.netflix.iceberg.util; +import com.google.common.base.Preconditions; +import com.google.common.collect.Iterables; +import com.netflix.iceberg.io.CloseableGroup; +import java.io.Closeable; import java.util.Iterator; import java.util.NoSuchElementException; import java.util.concurrent.ConcurrentLinkedQueue; -import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; import java.util.concurrent.Future; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.TimeoutException; -public class ParallelIterable implements Iterable { +public class ParallelIterable extends CloseableGroup implements Iterable { private final Iterable> iterables; - private final ExecutorService trackingPool; private final ExecutorService workerPool; public ParallelIterable(Iterable> iterables, - ExecutorService trackingPool, ExecutorService workerPool) { this.iterables = iterables; -this.trackingPool = trackingPool; this.workerPool = workerPool; } @Override public Iterator iterator() { -return new ParallelIterator<>(iterables, trackingPool, workerPool); +ParallelIterator iter = new ParallelIterator<>(iterables, workerPool); +addCloseable(iter); +return iter; } - private static class ParallelIterator implements Iterator { + private static class ParallelIterator implements Iterator, Closeable { +private final Iterator tasks; +private final ExecutorService workerPool; +private final Future[] taskFutures; private final ConcurrentLinkedQueue queue = new ConcurrentLinkedQueue<>(); -private final Future taskFuture; - -public ParallelIterator(Iterable> iterables, -ExecutorService trackingPool, -ExecutorService workerPool) { - this.taskFuture = trackingPool.submit(() -> { -Tasks.foreach(iterables) -.noRetry().stopOnFailure().throwFailureWhenFinished() -.executeWith(workerPool) -.run(iterable -> { - for (T item : iterable) { -queue.add(item); - } -}); -return true; - }); +private boolean closed = false; + +private ParallelIterator(Iterable> iterables, + ExecutorService workerPool) { + this.tasks = Iterables.transform(iterables, iterable -> + (Runnable) () -> { +for (T item : iterable) { + queue.add(item); +} + }).iterator(); + this.workerPool = workerPool; + // submit 2 tasks per worker at a time + this.taskFutures = new Future[2 * ThreadPools.WORKER_THREAD_POOL_SIZE]; +} + +@Override +public void close() { + // cancel background tasks + for (int i = 0; i < taskFutures.length; i += 1) { +if (taskFutures[i] != null && !taskFutures[i].isDone()) { + taskFutures[i].cancel(true); +} + } + this.closed = true; +} + +/** + * Checks on running tasks and submits new tasks if needed. + * + * This should not be called after {@link #close()}. + * + * @return true if there are pending tasks, false otherwise + */ +private boolean checkTasks() { + boolean hasRunningTask = false; + + for (int i = 0; i < taskFutures.length; i += 1) { +if (taskFutures[i] == null || taskFutures[i].isDone()) { + taskFutures[i] = submitNextTask(); +} + +if (taskFutures[i] != null) { + hasRunningTask = true; +} + } + + return tasks.hasNext() || hasRunningTask; +} + +private Future submitNextTask() { + if (tasks.hasNext()) { +return workerPool.submit(tasks.next()); + } + return null; } @Override public synchronized boolean hasNext() { + Preconditions.checkState(!closed, "Already closed"); + + // if the consumer is processing records more slowly than the producers, then this check will + // prevent tasks from being submitted. while the producers are running, this will always + // return here before running checkTasks. when enough of the tasks are finished that the + // consumer catches up, then lots of new tasks will be submitted at once. this behavior is + // okay because it ensures that records are not stacking up waiting to be consumed and taking + // up memory. + // + // consumers that process results quickly will periodically exhaust the queue and submit new + // tasks when checkTasks runs. fast consumers should not be delaye
[GitHub] rdsr commented on a change in pull request #45: Lazily submit tasks in ParallelIterable and add cancellation.
rdsr commented on a change in pull request #45: Lazily submit tasks in ParallelIterable and add cancellation. URL: https://github.com/apache/incubator-iceberg/pull/45#discussion_r240901429 ## File path: core/src/main/java/com/netflix/iceberg/util/ParallelIterable.java ## @@ -19,73 +19,124 @@ package com.netflix.iceberg.util; +import com.google.common.base.Preconditions; +import com.google.common.collect.Iterables; +import com.netflix.iceberg.io.CloseableGroup; +import java.io.Closeable; import java.util.Iterator; import java.util.NoSuchElementException; import java.util.concurrent.ConcurrentLinkedQueue; -import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; import java.util.concurrent.Future; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.TimeoutException; -public class ParallelIterable implements Iterable { +public class ParallelIterable extends CloseableGroup implements Iterable { private final Iterable> iterables; - private final ExecutorService trackingPool; private final ExecutorService workerPool; public ParallelIterable(Iterable> iterables, - ExecutorService trackingPool, ExecutorService workerPool) { this.iterables = iterables; -this.trackingPool = trackingPool; this.workerPool = workerPool; } @Override public Iterator iterator() { -return new ParallelIterator<>(iterables, trackingPool, workerPool); +ParallelIterator iter = new ParallelIterator<>(iterables, workerPool); +addCloseable(iter); +return iter; } - private static class ParallelIterator implements Iterator { + private static class ParallelIterator implements Iterator, Closeable { Review comment: I believe manual task tracking is required for cancellation of futures when the iterator is closed. Cancellation doesn't seem possible in this approach though, isn't? This is an automated message from the Apache Git Service. To respond to the message, please log on GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services
[GitHub] mccheah commented on a change in pull request #45: Lazily submit tasks in ParallelIterable and add cancellation.
mccheah commented on a change in pull request #45: Lazily submit tasks in ParallelIterable and add cancellation. URL: https://github.com/apache/incubator-iceberg/pull/45#discussion_r241103604 ## File path: core/src/main/java/com/netflix/iceberg/util/ParallelIterable.java ## @@ -19,73 +19,124 @@ package com.netflix.iceberg.util; +import com.google.common.base.Preconditions; +import com.google.common.collect.Iterables; +import com.netflix.iceberg.io.CloseableGroup; +import java.io.Closeable; import java.util.Iterator; import java.util.NoSuchElementException; import java.util.concurrent.ConcurrentLinkedQueue; -import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; import java.util.concurrent.Future; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.TimeoutException; -public class ParallelIterable implements Iterable { +public class ParallelIterable extends CloseableGroup implements Iterable { private final Iterable> iterables; - private final ExecutorService trackingPool; private final ExecutorService workerPool; public ParallelIterable(Iterable> iterables, - ExecutorService trackingPool, ExecutorService workerPool) { this.iterables = iterables; -this.trackingPool = trackingPool; this.workerPool = workerPool; } @Override public Iterator iterator() { -return new ParallelIterator<>(iterables, trackingPool, workerPool); +ParallelIterator iter = new ParallelIterator<>(iterables, workerPool); +addCloseable(iter); +return iter; } - private static class ParallelIterator implements Iterator { + private static class ParallelIterator implements Iterator, Closeable { Review comment: Sure it is - iterate through `runningTasks` and call `cancel` on everything. This is an automated message from the Apache Git Service. To respond to the message, please log on GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services
[GitHub] rdsr commented on a change in pull request #45: Lazily submit tasks in ParallelIterable and add cancellation.
rdsr commented on a change in pull request #45: Lazily submit tasks in ParallelIterable and add cancellation. URL: https://github.com/apache/incubator-iceberg/pull/45#discussion_r241119180 ## File path: core/src/main/java/com/netflix/iceberg/util/ParallelIterable.java ## @@ -19,73 +19,124 @@ package com.netflix.iceberg.util; +import com.google.common.base.Preconditions; +import com.google.common.collect.Iterables; +import com.netflix.iceberg.io.CloseableGroup; +import java.io.Closeable; import java.util.Iterator; import java.util.NoSuchElementException; import java.util.concurrent.ConcurrentLinkedQueue; -import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; import java.util.concurrent.Future; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.TimeoutException; -public class ParallelIterable implements Iterable { +public class ParallelIterable extends CloseableGroup implements Iterable { private final Iterable> iterables; - private final ExecutorService trackingPool; private final ExecutorService workerPool; public ParallelIterable(Iterable> iterables, - ExecutorService trackingPool, ExecutorService workerPool) { this.iterables = iterables; -this.trackingPool = trackingPool; this.workerPool = workerPool; } @Override public Iterator iterator() { -return new ParallelIterator<>(iterables, trackingPool, workerPool); +ParallelIterator iter = new ParallelIterator<>(iterables, workerPool); +addCloseable(iter); +return iter; } - private static class ParallelIterator implements Iterator { + private static class ParallelIterator implements Iterator, Closeable { Review comment: Thanks, I missed that! This is an automated message from the Apache Git Service. To respond to the message, please log on GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services
[GitHub] rdblue commented on issue #20: Encryption in Data Files
rdblue commented on issue #20: Encryption in Data Files URL: https://github.com/apache/incubator-iceberg/issues/20#issuecomment-446690252 @mccheah, can you also start a thread on the dev list to point out this spec? I think other people will probably be interested that aren't necessarily following the github issues. This is an automated message from the Apache Git Service. To respond to the message, please log on GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services
[GitHub] rdblue opened a new pull request #48: Fix commit retry with manfiest lists.
rdblue opened a new pull request #48: Fix commit retry with manfiest lists. URL: https://github.com/apache/incubator-iceberg/pull/48 A manifest list is created for every commit attempt. Before this update, the same file was used, which caused retries to fail trying to create the same list file. This uses a new location for every manifest list, keeps track of old lists, and cleans up unused lists after a commit succeeds. This is an automated message from the Apache Git Service. To respond to the message, please log on GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services
[GitHub] rdblue closed pull request #46: Do not scan manifests with no deletes when expiring snapshots.
rdblue closed pull request #46: Do not scan manifests with no deletes when expiring snapshots. URL: https://github.com/apache/incubator-iceberg/pull/46 This is a PR merged from a forked repository. As GitHub hides the original diff on merge, it is displayed below for the sake of provenance: As this is a foreign pull request (from a fork), the diff is supplied below (as it won't show otherwise due to GitHub magic): diff --git a/core/src/main/java/com/netflix/iceberg/RemoveSnapshots.java b/core/src/main/java/com/netflix/iceberg/RemoveSnapshots.java index 541cc5f..4784dd1 100644 --- a/core/src/main/java/com/netflix/iceberg/RemoveSnapshots.java +++ b/core/src/main/java/com/netflix/iceberg/RemoveSnapshots.java @@ -162,8 +162,11 @@ public void commit() { .onFailure((item, exc) -> LOG.warn("Failed to get deleted files: this may cause orphaned data files", exc) ).run(manifest -> { - // even if the manifest is still used, it may contain files that can be deleted - // TODO: eliminate manifests with no deletes without scanning + if (manifest.deletedFilesCount() != null && manifest.deletedFilesCount() == 0) { +return; + } + + // the manifest has deletes, scan it to find files to delete try (ManifestReader reader = ManifestReader.read(ops.io().newInputFile(manifest.path( { for (ManifestEntry entry : reader.entries()) { // if the snapshot ID of the DELETE entry is no longer valid, the data can be deleted This is an automated message from the Apache Git Service. To respond to the message, please log on GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services
[GitHub] vinooganesh commented on issue #23: DataFile External Identifier Field
vinooganesh commented on issue #23: DataFile External Identifier Field URL: https://github.com/apache/incubator-iceberg/issues/23#issuecomment-446758457 Hey @rdblue - quickly jumping in here. I think the mentality is that a file path as the sole identifier of a file may not suffice for every use case. Having an additional file identifier (independent of the physical path itself) would allow consumers of the system to both logically similar files and run operations on them. Specifically, let's say that I have something of a "source system" notion that I would want to persist on a per file basis. Having this state as an attribute on the File object itself would support this type of use case. Does that make sense? This is an automated message from the Apache Git Service. To respond to the message, please log on GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services
[GitHub] rdblue commented on issue #23: DataFile External Identifier Field
rdblue commented on issue #23: DataFile External Identifier Field URL: https://github.com/apache/incubator-iceberg/issues/23#issuecomment-446766094 @vinooganesh, I don't really understand the use case. How would you use the identifier? This is an automated message from the Apache Git Service. To respond to the message, please log on GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services
[GitHub] rdblue opened a new pull request #49: Fix type handling in Spark and Pig.
rdblue opened a new pull request #49: Fix type handling in Spark and Pig. URL: https://github.com/apache/incubator-iceberg/pull/49 This copies Pig type handling from Spark and fixes a minor bug in Spark with integer logical types that have been promoted to longs. This is an automated message from the Apache Git Service. To respond to the message, please log on GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services
[GitHub] groodt commented on issue #46: Do not scan manifests with no deletes when expiring snapshots.
groodt commented on issue #46: Do not scan manifests with no deletes when expiring snapshots. URL: https://github.com/apache/incubator-iceberg/pull/46#issuecomment-446770708 I'm sorry. I really don't wish to be annoying, but I'm getting spammed by something called GitBox for all activity on GitHub. I don't know where or how to disable it. Any ideas? I've unsubscribed from the Apache mailing list by sending an email to dev-unsubscr...@iceberg.apache.org That's the only place that I think I signed up. This is an automated message from the Apache Git Service. To respond to the message, please log on GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services
[GitHub] rdblue commented on issue #46: Do not scan manifests with no deletes when expiring snapshots.
rdblue commented on issue #46: Do not scan manifests with no deletes when expiring snapshots. URL: https://github.com/apache/incubator-iceberg/pull/46#issuecomment-446775717 @groodt, I think we've requested for gitbox notifications to stop going to the dev list, but I'll check on that. Unsubscribing from dev should fix it, I'm not sure where you're getting notifications from if you aren't subscribed to dev. This is an automated message from the Apache Git Service. To respond to the message, please log on GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services
[GitHub] rdblue commented on issue #46: Do not scan manifests with no deletes when expiring snapshots.
rdblue commented on issue #46: Do not scan manifests with no deletes when expiring snapshots. URL: https://github.com/apache/incubator-iceberg/pull/46#issuecomment-446781096 Looks like we had not requested it yet, we had asked a question about it. I've submitted the request: https://issues.apache.org/jira/browse/INFRA-17413 Thanks for saying something! This is an automated message from the Apache Git Service. To respond to the message, please log on GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services
[GitHub] rdblue opened a new pull request #50: Use manifest lists by default and fix tests.
rdblue opened a new pull request #50: Use manifest lists by default and fix tests. URL: https://github.com/apache/incubator-iceberg/pull/50 This also fixes tests that were picking up manifest list files because they validated manifests in the metadata directory by looking for all Avro files. Those tests now use `listManifestFiles` that will filter out manifest lists. This fixes a minor bug in Snapshot.cacheChanges, where manifests was accessed directly instead of using the lazy-loading method. This is an automated message from the Apache Git Service. To respond to the message, please log on GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services
[GitHub] danielcweeks commented on issue #49: Fix type handling in Spark and Pig.
danielcweeks commented on issue #49: Fix type handling in Spark and Pig. URL: https://github.com/apache/incubator-iceberg/pull/49#issuecomment-447036149 +1 This is an automated message from the Apache Git Service. To respond to the message, please log on GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services
[GitHub] rdblue closed pull request #49: Fix type handling in Spark and Pig.
rdblue closed pull request #49: Fix type handling in Spark and Pig. URL: https://github.com/apache/incubator-iceberg/pull/49 This is a PR merged from a forked repository. As GitHub hides the original diff on merge, it is displayed below for the sake of provenance: As this is a foreign pull request (from a fork), the diff is supplied below (as it won't show otherwise due to GitHub magic): diff --git a/pig/src/main/java/com/netflix/iceberg/pig/PigParquetReader.java b/pig/src/main/java/com/netflix/iceberg/pig/PigParquetReader.java index 1de637f..dbd9282 100644 --- a/pig/src/main/java/com/netflix/iceberg/pig/PigParquetReader.java +++ b/pig/src/main/java/com/netflix/iceberg/pig/PigParquetReader.java @@ -38,6 +38,7 @@ import com.netflix.iceberg.parquet.ParquetValueReaders.StructReader; import com.netflix.iceberg.parquet.ParquetValueReaders.UnboxedReader; import com.netflix.iceberg.parquet.TypeWithSchemaVisitor; +import com.netflix.iceberg.types.Type.TypeID; import com.netflix.iceberg.types.Types; import org.apache.parquet.column.ColumnDescriptor; import org.apache.parquet.schema.DecimalMetadata; @@ -208,8 +209,10 @@ public PigParquetReader(Schema readSchema, MessageType fileSchema, Map(desc); case TIMESTAMP_MILLIS: return new TimestampMillisReader(desc); @@ -234,13 +237,20 @@ public PigParquetReader(Schema readSchema, MessageType fileSchema, Map(desc); + } case FLOAT: - if(expected.typeId() == Types.DoubleType.get().typeId()) { + if (expected != null && expected.typeId() == TypeID.DOUBLE) { return new FloatAsDoubleReader(desc); + } else { +return new UnboxedReader<>(desc); } +case BOOLEAN: +case INT64: case DOUBLE: return new UnboxedReader<>(desc); default: diff --git a/spark/src/main/java/com/netflix/iceberg/spark/data/SparkParquetReaders.java b/spark/src/main/java/com/netflix/iceberg/spark/data/SparkParquetReaders.java index 1791014..336ffb1 100644 --- a/spark/src/main/java/com/netflix/iceberg/spark/data/SparkParquetReaders.java +++ b/spark/src/main/java/com/netflix/iceberg/spark/data/SparkParquetReaders.java @@ -204,10 +204,15 @@ private SparkParquetReaders() { case JSON: case UTF8: return new StringReader(desc); - case DATE: case INT_8: case INT_16: case INT_32: +if (expected != null && expected.typeId() == Types.LongType.get().typeId()) { + return new IntAsLongReader(desc); +} else { + return new UnboxedReader(desc); +} + case DATE: case INT_64: case TIMESTAMP_MICROS: return new UnboxedReader<>(desc); This is an automated message from the Apache Git Service. To respond to the message, please log on GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services
[GitHub] vinooganesh commented on issue #23: DataFile External Identifier Field
vinooganesh commented on issue #23: DataFile External Identifier Field URL: https://github.com/apache/incubator-iceberg/issues/23#issuecomment-447057968 So I see 2 uses for this: (1) Identifier shared across files - let's say that I have a bunch of files that make up a RDD that come from different systems (for example, let's say we're a bank and we have a bunch of customers from M&A, wealth management, etc..) and they each give us a list of their customers that we union together to make up the RDD. Let's say one of them is corrupt / doesn't work, and thus our RDD is in a bad state. Having this identifier would allow us to link the file to the source system the the file came from and allow us to talk to the data owners to remedy the issue. (2) Unique identifier on a per file basis - In this situation, we simply want a way to retrieve some static information on a per file basis outside of the path itself. For example, I think of this as something like the `Descriptor` in the SSTable object in Cassandra (https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/io/sstable/Descriptor.java#L62). The object does include the directory (https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/io/sstable/Descriptor.java#L56), but also includes something like the FormatType (https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/io/sstable/Descriptor.java#L62 - an enum for different SSTable formats). Taking a step back, it does sound a bit like metadata, but I do think having this type of information somewhere is important. Cassandra kind of hacks around it by encoding things like the SSTable version in the name itself (the path contains it), but we don't really have a similar way to retrieve this type of information without this identifier. This is an automated message from the Apache Git Service. To respond to the message, please log on GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services
[GitHub] rdblue opened a new pull request #51: Use manifest lists by default.
rdblue opened a new pull request #51: Use manifest lists by default. URL: https://github.com/apache/incubator-iceberg/pull/51 This also fixes tests that were broken by defaulting to manifest lists because they assumed that the only avro files in the metadata directory were manifests. This is an automated message from the Apache Git Service. To respond to the message, please log on GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services
[GitHub] rdblue commented on issue #50: Use manifest lists by default and fix tests.
rdblue commented on issue #50: Use manifest lists by default and fix tests. URL: https://github.com/apache/incubator-iceberg/pull/50#issuecomment-447117765 Accidentally duplicated this one in #51. Closing. This is an automated message from the Apache Git Service. To respond to the message, please log on GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services
[GitHub] rdblue commented on issue #51: Use manifest lists by default.
rdblue commented on issue #51: Use manifest lists by default. URL: https://github.com/apache/incubator-iceberg/pull/51#issuecomment-447117967 Test failures are because #48 is missing. This is an automated message from the Apache Git Service. To respond to the message, please log on GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services
[GitHub] rdblue closed pull request #50: Use manifest lists by default and fix tests.
rdblue closed pull request #50: Use manifest lists by default and fix tests. URL: https://github.com/apache/incubator-iceberg/pull/50 This is a PR merged from a forked repository. As GitHub hides the original diff on merge, it is displayed below for the sake of provenance: As this is a foreign pull request (from a fork), the diff is supplied below (as it won't show otherwise due to GitHub magic): diff --git a/api/src/main/java/com/netflix/iceberg/Files.java b/api/src/main/java/com/netflix/iceberg/Files.java index e85825a..197dcc1 100644 --- a/api/src/main/java/com/netflix/iceberg/Files.java +++ b/api/src/main/java/com/netflix/iceberg/Files.java @@ -99,6 +99,9 @@ public static InputFile localInput(File file) { } public static InputFile localInput(String file) { +if (file.startsWith("file:")) { + return localInput(new File(file.replaceFirst("file:", ""))); +} return localInput(new File(file)); } diff --git a/core/src/main/java/com/netflix/iceberg/BaseSnapshot.java b/core/src/main/java/com/netflix/iceberg/BaseSnapshot.java index 36a873a..554f24f 100644 --- a/core/src/main/java/com/netflix/iceberg/BaseSnapshot.java +++ b/core/src/main/java/com/netflix/iceberg/BaseSnapshot.java @@ -138,7 +138,7 @@ private void cacheChanges() { // accumulate adds and deletes from all manifests. // because manifests can be reused in newer snapshots, filter the changes by snapshot id. -for (String manifest : Iterables.transform(manifests, ManifestFile::path)) { +for (String manifest : Iterables.transform(manifests(), ManifestFile::path)) { try (ManifestReader reader = ManifestReader.read(ops.io().newInputFile(manifest))) { for (ManifestEntry add : reader.addedFiles()) { if (add.snapshotId() == snapshotId) { @@ -164,7 +164,7 @@ public String toString() { return Objects.toStringHelper(this) .add("id", snapshotId) .add("timestamp_ms", timestampMillis) -.add("manifests", manifests) +.add("manifests", manifests()) .toString(); } } diff --git a/core/src/main/java/com/netflix/iceberg/TableProperties.java b/core/src/main/java/com/netflix/iceberg/TableProperties.java index 0d99c7e..69bfcf2 100644 --- a/core/src/main/java/com/netflix/iceberg/TableProperties.java +++ b/core/src/main/java/com/netflix/iceberg/TableProperties.java @@ -73,5 +73,5 @@ public static final String WRITE_NEW_DATA_LOCATION = "write.folder-storage.path"; public static final String MANIFEST_LISTS_ENABLED = "write.manifest-lists.enabled"; - public static final boolean MANIFEST_LISTS_ENABLED_DEFAULT = false; + public static final boolean MANIFEST_LISTS_ENABLED_DEFAULT = true; } diff --git a/core/src/test/java/com/netflix/iceberg/TableTestBase.java b/core/src/test/java/com/netflix/iceberg/TableTestBase.java index c723daa..010896c 100644 --- a/core/src/test/java/com/netflix/iceberg/TableTestBase.java +++ b/core/src/test/java/com/netflix/iceberg/TableTestBase.java @@ -94,13 +94,13 @@ public void cleanupTables() { TestTables.clearTables(); } - List listMetadataFiles(String ext) { -return listMetadataFiles(tableDir, ext); + List listManifestFiles() { +return listManifestFiles(tableDir); } - List listMetadataFiles(File tableDir, String ext) { -return Lists.newArrayList(new File(tableDir, "metadata").listFiles( -(dir, name) -> Files.getFileExtension(name).equalsIgnoreCase(ext))); + List listManifestFiles(File tableDir) { +return Lists.newArrayList(new File(tableDir, "metadata").listFiles((dir, name) -> +!name.startsWith("snap") && Files.getFileExtension(name).equalsIgnoreCase("avro"))); } private TestTables.TestTable create(Schema schema, PartitionSpec spec) { diff --git a/core/src/test/java/com/netflix/iceberg/TestCreateTransaction.java b/core/src/test/java/com/netflix/iceberg/TestCreateTransaction.java index 257f6e3..ffdec59 100644 --- a/core/src/test/java/com/netflix/iceberg/TestCreateTransaction.java +++ b/core/src/test/java/com/netflix/iceberg/TestCreateTransaction.java @@ -49,7 +49,7 @@ public void testCreateTransaction() throws IOException { Assert.assertEquals("Should have metadata version 0", 0, (int) TestTables.metadataVersion("test_create")); Assert.assertEquals("Should have 0 manifest files", -0, listMetadataFiles(tableDir, "avro").size()); +0, listManifestFiles(tableDir).size()); Assert.assertEquals("Table schema should match with reassigned IDs", assignFreshIds(SCHEMA).asStruct(), meta.schema().asStruct()); @@ -86,7 +86,7 @@ public void testCreateAndAppendWithTransaction() throws IOException { Assert.assertEquals("Should have metadata version 0", 0, (int) TestTables.metadataVersion("test_append")); Assert.assertEquals("Should have 1 manifest file", -1, listMetadataFiles(tableDir, "avro").size()); +1, listManifestFiles(tableDir).size()); Assert.assertEqua
[GitHub] danielcweeks commented on issue #48: Fix commit retry with manfiest lists.
danielcweeks commented on issue #48: Fix commit retry with manfiest lists. URL: https://github.com/apache/incubator-iceberg/pull/48#issuecomment-447118392 +1 This is an automated message from the Apache Git Service. To respond to the message, please log on GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services
[GitHub] rdblue closed pull request #48: Fix commit retry with manfiest lists.
rdblue closed pull request #48: Fix commit retry with manfiest lists. URL: https://github.com/apache/incubator-iceberg/pull/48 This is a PR merged from a forked repository. As GitHub hides the original diff on merge, it is displayed below for the sake of provenance: As this is a foreign pull request (from a fork), the diff is supplied below (as it won't show otherwise due to GitHub magic): diff --git a/core/src/main/java/com/netflix/iceberg/SnapshotUpdate.java b/core/src/main/java/com/netflix/iceberg/SnapshotUpdate.java index ce9d59c..796df2f 100644 --- a/core/src/main/java/com/netflix/iceberg/SnapshotUpdate.java +++ b/core/src/main/java/com/netflix/iceberg/SnapshotUpdate.java @@ -22,6 +22,7 @@ import com.google.common.cache.CacheBuilder; import com.google.common.cache.CacheLoader; import com.google.common.cache.LoadingCache; +import com.google.common.collect.Lists; import com.google.common.collect.Sets; import com.netflix.iceberg.exceptions.CommitFailedException; import com.netflix.iceberg.exceptions.RuntimeIOException; @@ -35,6 +36,7 @@ import java.util.List; import java.util.Set; import java.util.UUID; +import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; import static com.netflix.iceberg.TableProperties.COMMIT_MAX_RETRY_WAIT_MS; @@ -70,6 +72,8 @@ public ManifestFile load(ManifestFile file) { private final TableOperations ops; private final String commitUUID = UUID.randomUUID().toString(); + private final AtomicInteger attempt = new AtomicInteger(0); + private final List manifestLists = Lists.newArrayList(); private Long snapshotId = null; private TableMetadata base = null; @@ -110,7 +114,11 @@ public Snapshot apply() { OutputFile manifestList = manifestListPath(); try (ManifestListWriter writer = new ManifestListWriter( - manifestListPath(), snapshotId(), parentSnapshotId)) { + manifestList, snapshotId(), parentSnapshotId)) { + +// keep track of the manifest lists created +manifestLists.add(manifestList.location()); + ManifestFile[] manifestFiles = new ManifestFile[manifests.size()]; Tasks.range(manifestFiles.length) @@ -172,6 +180,12 @@ public void commit() { Snapshot saved = ops.refresh().snapshot(newSnapshotId.get()); if (saved != null) { cleanUncommitted(Sets.newHashSet(saved.manifests())); +// also clean up unused manifest lists created by multiple attempts +for (String manifestList : manifestLists) { + if (!saved.manifestListLocation().equals(manifestList)) { +ops.io().deleteFile(manifestList); + } +} } else { // saved may not be present if the latest metadata couldn't be loaded due to eventual // consistency problems in refresh. in that case, don't clean up. @@ -184,6 +198,10 @@ public void commit() { } protected void cleanAll() { +for (String manifestList : manifestLists) { + ops.io().deleteFile(manifestList); +} +manifestLists.clear(); cleanUncommitted(EMPTY_SET); } @@ -193,7 +211,7 @@ protected void deleteFile(String path) { protected OutputFile manifestListPath() { return ops.io().newOutputFile(ops.metadataFileLocation(FileFormat.AVRO.addExtension( -String.format("snap-%d-%s", snapshotId(), commitUUID; +String.format("snap-%d-%d-%s", snapshotId(), attempt.incrementAndGet(), commitUUID; } protected OutputFile manifestPath(int i) { diff --git a/core/src/test/java/com/netflix/iceberg/TestFastAppend.java b/core/src/test/java/com/netflix/iceberg/TestFastAppend.java index 4d9e174..88252bb 100644 --- a/core/src/test/java/com/netflix/iceberg/TestFastAppend.java +++ b/core/src/test/java/com/netflix/iceberg/TestFastAppend.java @@ -171,7 +171,32 @@ public void testFailure() { } @Test - public void testRecovery() { + public void testRecoveryWithManifestList() { +table.updateProperties().set(TableProperties.MANIFEST_LISTS_ENABLED, "true").commit(); + +// inject 3 failures, the last try will succeed +TestTables.TestTableOperations ops = table.ops(); +ops.failCommits(3); + +AppendFiles append = table.newFastAppend().appendFile(FILE_B); +Snapshot pending = append.apply(); +ManifestFile newManifest = pending.manifests().get(0); +Assert.assertTrue("Should create new manifest", new File(newManifest.path()).exists()); + +append.commit(); + +TableMetadata metadata = readMetadata(); + +validateSnapshot(null, metadata.currentSnapshot(), FILE_B); +Assert.assertTrue("Should commit same new manifest", new File(newManifest.path()).exists()); +Assert.assertTrue("Should commit the same new manifest", +metadata.currentSnapshot().manifests().contains(newManifest)); + } + + @Test + public void testRecoveryWithoutManifestList() { +table.updateProperties().set(TableProperties.MANIFEST_LISTS_ENABLED, "false").com
[GitHub] rdblue commented on issue #28: Apply baseline checkstyle for iceberg-api only
rdblue commented on issue #28: Apply baseline checkstyle for iceberg-api only URL: https://github.com/apache/incubator-iceberg/pull/28#issuecomment-447124528 I think we should get this in, but I'd like to try to do it when we move from com.netflix packages to org.apache. That is going to touch every file as well, so we may as well do it at the same time. This is an automated message from the Apache Git Service. To respond to the message, please log on GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services
[GitHub] mccheah opened a new pull request #52: Use the FileIO submodule in Spark writers and readers.
mccheah opened a new pull request #52: Use the FileIO submodule in Spark writers and readers. URL: https://github.com/apache/incubator-iceberg/pull/52 Tricky because the table operations needs to be exposed. Added a mixed interface of Table + TableOperations accordingly. Therefore now the Spark data source is opinionated about ensuring the returned table also implements `HasTableOperations`. This is an automated message from the Apache Git Service. To respond to the message, please log on GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services
[GitHub] mccheah commented on a change in pull request #52: Use the FileIO submodule in Spark writers and readers.
mccheah commented on a change in pull request #52: Use the FileIO submodule in Spark writers and readers. URL: https://github.com/apache/incubator-iceberg/pull/52#discussion_r241580782 ## File path: spark/src/main/java/com/netflix/iceberg/spark/source/IcebergSource.java ## @@ -89,10 +90,10 @@ public DataSourceReader createReader(DataSourceOptions options) { .toUpperCase(Locale.ENGLISH)); } -return Optional.of(new Writer(table, lazyConf(), format)); +return Optional.of(new Writer(table, format, table.operations().io())); } - protected Table findTable(DataSourceOptions options) { + protected TableWithTableOperations findTable(DataSourceOptions options) { Review comment: @rdblue this is the most interesting question raised by this patch. What do you think? This is an automated message from the Apache Git Service. To respond to the message, please log on GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services
[GitHub] rdblue commented on a change in pull request #52: Use the FileIO submodule in Spark writers and readers.
rdblue commented on a change in pull request #52: Use the FileIO submodule in Spark writers and readers. URL: https://github.com/apache/incubator-iceberg/pull/52#discussion_r241587801 ## File path: spark/src/main/java/com/netflix/iceberg/spark/source/IcebergSource.java ## @@ -89,10 +90,10 @@ public DataSourceReader createReader(DataSourceOptions options) { .toUpperCase(Locale.ENGLISH)); } -return Optional.of(new Writer(table, lazyConf(), format)); +return Optional.of(new Writer(table, format, table.operations().io())); } - protected Table findTable(DataSourceOptions options) { + protected TableWithTableOperations findTable(DataSourceOptions options) { Review comment: What about adding `io` to the `Table` interface? I'd rather do that since `FileIO` is a public interface. I think that is mostly what HasTableOperations is used for anyway. This is an automated message from the Apache Git Service. To respond to the message, please log on GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services
[GitHub] mccheah commented on a change in pull request #52: Use the FileIO submodule in Spark writers and readers.
mccheah commented on a change in pull request #52: Use the FileIO submodule in Spark writers and readers. URL: https://github.com/apache/incubator-iceberg/pull/52#discussion_r241604276 ## File path: spark/src/main/java/com/netflix/iceberg/spark/source/IcebergSource.java ## @@ -89,10 +90,10 @@ public DataSourceReader createReader(DataSourceOptions options) { .toUpperCase(Locale.ENGLISH)); } -return Optional.of(new Writer(table, lazyConf(), format)); +return Optional.of(new Writer(table, format, table.operations().io())); } - protected Table findTable(DataSourceOptions options) { + protected TableWithTableOperations findTable(DataSourceOptions options) { Review comment: I think that's fine. This is an automated message from the Apache Git Service. To respond to the message, please log on GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services
[GitHub] mccheah commented on a change in pull request #52: Use the FileIO submodule in Spark writers and readers.
mccheah commented on a change in pull request #52: Use the FileIO submodule in Spark writers and readers. URL: https://github.com/apache/incubator-iceberg/pull/52#discussion_r241606297 ## File path: api/src/main/java/com/netflix/iceberg/Table.java ## @@ -171,4 +172,10 @@ default AppendFiles newFastAppend() { * @return a new {@link Transaction} */ Transaction newTransaction(); + + /** + * @return a {@link FileIO} to read and write table data and metadata files + */ + FileIO io(); Review comment: Should `TableOperations#io` continue to exist? This is an automated message from the Apache Git Service. To respond to the message, please log on GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services
[GitHub] [iceberg-docs] samredai opened a new pull request #1: First version of hugo doc site
samredai opened a new pull request #1: URL: https://github.com/apache/iceberg-docs/pull/1 This is the hugo docs-site shared on the dev mailing list and in [apache/iceberg/issues/3616](https://github.com/apache/iceberg/issues/3616). It includes a github workflow that builds and pushes the site to the `asf-site` branch in a subdirectory named after the version. These sites are accessed via the "Releases" navigation menu section and currently I've added `0.12.0`, `0.12.1`, and `next` (we just have to push to branches with the same name). We also need to push to a `latest` branch which the root site automatically redirects to (also driven by the github workflow and a meta tag in the `redirect/index.html` file). A follow-up item is including some kind of scheduled automated push (daily? weekly?) from the main iceberg repo to the main branch in this repo. This will keep a `/main/docs` version of the site that always up to date with the apache/iceberg/master branch. Thinking about this now, we could probably get rid of the main branch here and set `next` as the default branch for the repo, or alternatively we can have some "if main then next" logic in a couple of places. -- 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: dev-unsubscr...@iceberg.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [iceberg-docs] samredai commented on pull request #1: First version of hugo doc site
samredai commented on pull request #1: URL: https://github.com/apache/iceberg-docs/pull/1#issuecomment-986105830 @rdblue @jackye1995 @kbendick @ajantha-bhat @RussellSpitzer @ -- 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: dev-unsubscr...@iceberg.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [iceberg-docs] samredai edited a comment on pull request #1: First version of hugo doc site
samredai edited a comment on pull request #1: URL: https://github.com/apache/iceberg-docs/pull/1#issuecomment-986105830 @rdblue @jackye1995 @kbendick @ajantha-bhat @RussellSpitzer -- 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: dev-unsubscr...@iceberg.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [iceberg-docs] jackye1995 commented on pull request #1: First version of hugo doc site
jackye1995 commented on pull request #1: URL: https://github.com/apache/iceberg-docs/pull/1#issuecomment-986153886 There are over 3600 files, what are the important files to look into? -- 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: dev-unsubscr...@iceberg.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [iceberg-docs] rdblue commented on pull request #1: First version of hugo doc site
rdblue commented on pull request #1: URL: https://github.com/apache/iceberg-docs/pull/1#issuecomment-986315139 Yeah, could you break this into smaller commits that we can look at? Why is this almost 3,700 files? -- 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: dev-unsubscr...@iceberg.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [iceberg-docs] samredai commented on pull request #1: First version of hugo doc site
samredai commented on pull request #1: URL: https://github.com/apache/iceberg-docs/pull/1#issuecomment-986319585 Ah, my bad that's from two things: - Didn't realize I included all of the javadocs, (which accounts for 3,310 files) - I included the theme repo in this actual repo but it can be added as a submodule as described [here](https://github.com/alex-shpak/hugo-book#install-as-git-submodule) (which accounts for 177 files) Converting this to a draft for now while I fix that. -- 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: dev-unsubscr...@iceberg.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [iceberg-docs] rdblue commented on pull request #1: First version of hugo doc site
rdblue commented on pull request #1: URL: https://github.com/apache/iceberg-docs/pull/1#issuecomment-986321968 Makes sense! Let's remove whatever we can and then follow up with Javadocs in a separate commit for each version once we have the basics. That should let us review it much more easily. Thanks, @samredai. -- 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: dev-unsubscr...@iceberg.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [iceberg-docs] samredai commented on pull request #1: First version of hugo doc site
samredai commented on pull request #1: URL: https://github.com/apache/iceberg-docs/pull/1#issuecomment-986415495 Updated to not include the javadocs and also to include the theme as a submodule. An overview of the important directories/files. - `content/docs` directory includes all of the markdown pages - `config.toml` includes the hugo site configuration - `.github/workflows/deploy.yml` includes the github workflow for deploying versioned sites to the `asf-site` branch - `asciinema` directory includes a README and a helper python script to easily generate asciinema recordings -- 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: dev-unsubscr...@iceberg.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [iceberg-docs] jackye1995 commented on a change in pull request #1: First version of hugo doc site
jackye1995 commented on a change in pull request #1: URL: https://github.com/apache/iceberg-docs/pull/1#discussion_r762692930 ## File path: .github/workflows/deploy.yml ## @@ -0,0 +1,52 @@ +name: github pages + +on: [push, pull_request] + +jobs: + deploy: +runs-on: ubuntu-20.04 +steps: + - uses: actions/checkout@v2 + + - name: Set output +id: vars +run: echo ::set-output name=branch_name::${GITHUB_REF#refs/*/} + + - name: Set baseURL in config.toml +uses: ci/toml-editor@1.0.0 +with: + file: "config.toml" + key: "baseURL" + value: "https://samredai.github.io/iceberg-docs-prototype/${{ steps.vars.outputs.branch_name }}" Review comment: needs to be updated? ## File path: README.md ## @@ -1,3 +1,19 @@ -## Iceberg Docs +# Apache Iceberg Documentation Site -This repository contains the markdown documentation hosted at https://iceberg.apache.org. +This repository contains the documentation for [Apache Iceberg](https://github.com/apache/iceberg). +It's built with [Hugo](https://gohugo.io/) and hosted on GitHub pages. Review comment: hosted at https://iceberg.apache.org ? ## File path: config.toml ## @@ -0,0 +1,24 @@ +baseURL = "" # This is populated by the github deploy workflow and is equal to "/" +languageCode = "en-us" +title = "Apache Iceberg" +theme= "hugo-book" + +[params] + BookLogo = "img/iceberg-logo-icon.png" + description = "The open table format for analytic datasets." + versions.iceberg = "" # This is populated by the github deploy workflow and is equal to the branch name + versions.nessie = "0.15.1" + latestVersions.iceberg = "0.12.1" # This is used for the version badge on the "latest" site version + +[[params.social]] + title = "devlist" Review comment: indentation seems off here? ## File path: asciinema/schema_evolution.py ## @@ -0,0 +1,53 @@ +from generate_asciinema_cast import Cast + +sequence = [ +( +"ALTER TABLE taxis ADD COLUMN fare_per_distance_unit float AFTER trip_distance;", +"Time taken: 0.671 seconds", +), +( + "DESCRIBE TABLE nyc.taxis;", + """VendorIDstring +tpep_pickup_datetimestring Review comment: indentation seems off here? -- 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: dev-unsubscr...@iceberg.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [iceberg-docs] rdblue commented on a change in pull request #1: First version of hugo doc site
rdblue commented on a change in pull request #1: URL: https://github.com/apache/iceberg-docs/pull/1#discussion_r763192839 ## File path: README.md ## @@ -1,3 +1,19 @@ -## Iceberg Docs +# Apache Iceberg Documentation Site -This repository contains the markdown documentation hosted at https://iceberg.apache.org. +This repository contains the documentation for [Apache Iceberg](https://github.com/apache/iceberg). +It's built with [Hugo](https://gohugo.io/) and hosted on GitHub pages. Review comment: Yeah, I think that it is actually hosted by ASF infra. -- 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: dev-unsubscr...@iceberg.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [iceberg-docs] rdblue commented on a change in pull request #1: First version of hugo doc site
rdblue commented on a change in pull request #1: URL: https://github.com/apache/iceberg-docs/pull/1#discussion_r763195072 ## File path: asciinema/schema_evolution.py ## @@ -0,0 +1,53 @@ +from generate_asciinema_cast import Cast + +sequence = [ +( +"ALTER TABLE taxis ADD COLUMN fare_per_distance_unit float AFTER trip_distance;", +"Time taken: 0.671 seconds", +), +( + "DESCRIBE TABLE nyc.taxis;", + """VendorIDstring +tpep_pickup_datetimestring Review comment: I think this is to avoid indentation in the cast -- 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: dev-unsubscr...@iceberg.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [iceberg-docs] rdblue commented on a change in pull request #1: First version of hugo doc site
rdblue commented on a change in pull request #1: URL: https://github.com/apache/iceberg-docs/pull/1#discussion_r763195862 ## File path: config.toml ## @@ -0,0 +1,24 @@ +baseURL = "" # This is populated by the github deploy workflow and is equal to "/" +languageCode = "en-us" +title = "Apache Iceberg" +theme= "hugo-book" + +[params] + BookLogo = "img/iceberg-logo-icon.png" + description = "The open table format for analytic datasets." + versions.iceberg = "" # This is populated by the github deploy workflow and is equal to the branch name + versions.nessie = "0.15.1" + latestVersions.iceberg = "0.12.1" # This is used for the version badge on the "latest" site version + +[[params.social]] + title = "devlist" + icon = "envelope-o" + url = "mailto:dev-subscr...@iceberg.apache.org"; Review comment: Rather than linking to the dev list, what about linking to the community page that has a lot more information on the various ways to get involved? -- 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: dev-unsubscr...@iceberg.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [iceberg-docs] rdblue commented on a change in pull request #1: First version of hugo doc site
rdblue commented on a change in pull request #1: URL: https://github.com/apache/iceberg-docs/pull/1#discussion_r763196285 ## File path: content/about/about.md ## @@ -0,0 +1,9 @@ +--- +Title: What is Iceberg? +Draft: false +--- + +Iceberg adds tables to compute engines including Spark, Trino, PrestoDB, Flink and Hive using a high-performance table format that works just like a SQL table. +and avoids unpleasant surprises. Review comment: Typo? Looks like a sentence fragment left over from an edit. -- 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: dev-unsubscr...@iceberg.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [iceberg-docs] samredai commented on a change in pull request #1: First version of hugo doc site
samredai commented on a change in pull request #1: URL: https://github.com/apache/iceberg-docs/pull/1#discussion_r763652648 ## File path: .github/workflows/deploy.yml ## @@ -0,0 +1,52 @@ +name: github pages + +on: [push, pull_request] + +jobs: + deploy: +runs-on: ubuntu-20.04 +steps: + - uses: actions/checkout@v2 + + - name: Set output +id: vars +run: echo ::set-output name=branch_name::${GITHUB_REF#refs/*/} + + - name: Set baseURL in config.toml +uses: ci/toml-editor@1.0.0 +with: + file: "config.toml" + key: "baseURL" + value: "https://samredai.github.io/iceberg-docs-prototype/${{ steps.vars.outputs.branch_name }}" Review comment: Fixed! -- 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: dev-unsubscr...@iceberg.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [iceberg-docs] samredai commented on a change in pull request #1: First version of hugo doc site
samredai commented on a change in pull request #1: URL: https://github.com/apache/iceberg-docs/pull/1#discussion_r763652888 ## File path: README.md ## @@ -1,3 +1,19 @@ -## Iceberg Docs +# Apache Iceberg Documentation Site -This repository contains the markdown documentation hosted at https://iceberg.apache.org. +This repository contains the documentation for [Apache Iceberg](https://github.com/apache/iceberg). +It's built with [Hugo](https://gohugo.io/) and hosted on GitHub pages. Review comment: Fixed! -- 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: dev-unsubscr...@iceberg.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [iceberg-docs] samredai commented on a change in pull request #1: First version of hugo doc site
samredai commented on a change in pull request #1: URL: https://github.com/apache/iceberg-docs/pull/1#discussion_r763656608 ## File path: config.toml ## @@ -0,0 +1,24 @@ +baseURL = "" # This is populated by the github deploy workflow and is equal to "/" +languageCode = "en-us" +title = "Apache Iceberg" +theme= "hugo-book" + +[params] + BookLogo = "img/iceberg-logo-icon.png" + description = "The open table format for analytic datasets." + versions.iceberg = "" # This is populated by the github deploy workflow and is equal to the branch name + versions.nessie = "0.15.1" + latestVersions.iceberg = "0.12.1" # This is used for the version badge on the "latest" site version + +[[params.social]] + title = "devlist" Review comment: Fixed! -- 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: dev-unsubscr...@iceberg.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [iceberg-docs] samredai commented on a change in pull request #1: First version of hugo doc site
samredai commented on a change in pull request #1: URL: https://github.com/apache/iceberg-docs/pull/1#discussion_r763658510 ## File path: config.toml ## @@ -0,0 +1,24 @@ +baseURL = "" # This is populated by the github deploy workflow and is equal to "/" +languageCode = "en-us" +title = "Apache Iceberg" +theme= "hugo-book" + +[params] + BookLogo = "img/iceberg-logo-icon.png" + description = "The open table format for analytic datasets." + versions.iceberg = "" # This is populated by the github deploy workflow and is equal to the branch name + versions.nessie = "0.15.1" + latestVersions.iceberg = "0.12.1" # This is used for the version badge on the "latest" site version + +[[params.social]] + title = "devlist" + icon = "envelope-o" + url = "mailto:dev-subscr...@iceberg.apache.org"; Review comment: Good idea, updated it to link to the Mailing List portion of the community page and updated the title to "Mailing Lists" -- 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: dev-unsubscr...@iceberg.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [iceberg-docs] samredai commented on a change in pull request #1: First version of hugo doc site
samredai commented on a change in pull request #1: URL: https://github.com/apache/iceberg-docs/pull/1#discussion_r763658651 ## File path: content/about/about.md ## @@ -0,0 +1,9 @@ +--- +Title: What is Iceberg? +Draft: false +--- + +Iceberg adds tables to compute engines including Spark, Trino, PrestoDB, Flink and Hive using a high-performance table format that works just like a SQL table. +and avoids unpleasant surprises. Review comment: Fixed! -- 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: dev-unsubscr...@iceberg.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [iceberg-docs] samredai commented on pull request #1: First version of hugo doc site
samredai commented on pull request #1: URL: https://github.com/apache/iceberg-docs/pull/1#issuecomment-987600519 > It looks like the Apache 2.0 license header is missing for some README and script files, please check and add, thanks! > > For Python script, it seems like indentation is off at some places. If we do not set up lint in this PR, please make sure we have a task to set it up later. Thanks for catching this! I added the Apache 2.0 license headers everywhere. I did a quick auto format of the python scripts with black. I'll follow up this PR with a tox file. -- 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: dev-unsubscr...@iceberg.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [iceberg-docs] kbendick commented on pull request #1: First version of hugo doc site
kbendick commented on pull request #1: URL: https://github.com/apache/iceberg-docs/pull/1#issuecomment-987616428 > Thinking about this now, we could probably get rid of the main branch here and set next as the default branch for the repo, or alternatively we can have some "if main then next" logic in a couple of places. I would be in favor of changing the "main" branch name to `next` or whatever we use for latest snapshot. Having the logical symmetry between the branch names for the versions and the branch name for `next` seems worth it to me (like actively maintained tags). If doing so provides context AND potentially reduces the need for defensive coding checks at all, I'm all for it. The easier the docs are to work with, the better they'll be. We can always revert if we don't care for it, but it's probably better to start earlier than well after the fact. -- 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: dev-unsubscr...@iceberg.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [iceberg-docs] jackye1995 commented on pull request #1: First version of hugo doc site
jackye1995 commented on pull request #1: URL: https://github.com/apache/iceberg-docs/pull/1#issuecomment-988424006 How are we going to move forward with this PR? If we move to the `next` branch, is this PR for the `main` branch still needed? -- 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: dev-unsubscr...@iceberg.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [iceberg-docs] samredai commented on pull request #1: First version of hugo doc site
samredai commented on pull request #1: URL: https://github.com/apache/iceberg-docs/pull/1#issuecomment-989259177 > How are we going to move forward with this PR? If we move to the `next` branch, is this PR for the `main` branch still needed? If a `next` branch is created I can re-open this PR against that. Or alternatively we could merge this, open a PR `main -> next`, merge that, set `next` as the default branch and then delete `main`. We would also have to remove the `main` directory from the `asf-site` branch, I can follow up with a PR for that. -- 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: dev-unsubscr...@iceberg.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [iceberg-docs] rdblue commented on a change in pull request #1: First version of hugo doc site
rdblue commented on a change in pull request #1: URL: https://github.com/apache/iceberg-docs/pull/1#discussion_r765314353 ## File path: config.toml ## @@ -0,0 +1,24 @@ +baseURL = "" # This is populated by the github deploy workflow and is equal to "/" +languageCode = "en-us" +title = "Apache Iceberg" +theme= "hugo-book" + +[params] + BookLogo = "img/iceberg-logo-icon.png" + description = "The open table format for analytic datasets." + versions.iceberg = "" # This is populated by the github deploy workflow and is equal to the branch name + versions.nessie = "0.15.1" + latestVersions.iceberg = "0.12.1" # This is used for the version badge on the "latest" site version + +[[params.social]] + title = "devlist" + icon = "envelope-o" + url = "mailto:dev-subscr...@iceberg.apache.org"; Review comment: Why not just a "Community" link? -- 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: dev-unsubscr...@iceberg.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [iceberg-docs] rdblue commented on a change in pull request #1: First version of hugo doc site
rdblue commented on a change in pull request #1: URL: https://github.com/apache/iceberg-docs/pull/1#discussion_r765318054 ## File path: config.toml ## @@ -0,0 +1,24 @@ +baseURL = "" # This is populated by the github deploy workflow and is equal to "/" +languageCode = "en-us" +title = "Apache Iceberg" +theme= "hugo-book" + +[params] + BookLogo = "img/iceberg-logo-icon.png" + description = "The open table format for analytic datasets." + versions.iceberg = "" # This is populated by the github deploy workflow and is equal to the branch name + versions.nessie = "0.15.1" + latestVersions.iceberg = "0.12.1" # This is used for the version badge on the "latest" site version + +[[params.social]] + title = "Mailing Lists" + icon = "envelope-o" + url = "https://iceberg.apache.org/#community/#mailing-lists"; Review comment: This link isn't correct. The hugo site won't use `#community`, it will use the old paths with no `#`. That's a good thing, since that matches the URLs that we've supported for a few years. -- 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: dev-unsubscr...@iceberg.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [iceberg-docs] rdblue commented on a change in pull request #1: First version of hugo doc site
rdblue commented on a change in pull request #1: URL: https://github.com/apache/iceberg-docs/pull/1#discussion_r765318518 ## File path: content/docs/_index.md ## @@ -0,0 +1,54 @@ +--- +title: Introduction +type: docs +--- + + + + +**Apache Iceberg is an open table format for huge analytic datasets.** Iceberg adds tables to compute engines including Spark, Trino, PrestoDB, Flink and Hive using a high-performance table format that works just like a SQL table. + +### User experience + +Iceberg avoids unpleasant surprises. Schema evolution works and won't inadvertently un-delete data. Users don't need to know about partitioning to get fast queries. + +* [Schema evolution](evolution#schema-evolution) supports add, drop, update, or rename, and has [no side-effects](evolution#correctness) Review comment: These links are broken for me. This takes me to /docs/evolution#schema-evolution. -- 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: dev-unsubscr...@iceberg.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [iceberg-docs] rdblue commented on pull request #1: First version of hugo doc site
rdblue commented on pull request #1: URL: https://github.com/apache/iceberg-docs/pull/1#issuecomment-989309666 It looks like this isn't quite ready to merge. Most of the text links on pages are broken and the nav menu is really huge. Nav links go to paths like `/spark-ddl/` but I think we probably want to change those to be `/0.12.1/spark-ddl/` for any version-specific pages and `/docs/` or `/community/` for top-level pages. Is it possible to get all the links working with that structure? Would we need to use relative links within pages like `/0.12.1/spark-ddl/` to get to pages under the same version without specifying the version in each link? -- 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: dev-unsubscr...@iceberg.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [iceberg-docs] rdblue commented on pull request #1: First version of hugo doc site
rdblue commented on pull request #1: URL: https://github.com/apache/iceberg-docs/pull/1#issuecomment-989311369 For the conversation about a `next` branch: * I don't see much value in having docs for the upcoming release. Maybe it's just me, but I'd not prioritize this very highly. * I'd focus on getting just 0.12.1 done. We can add more versions later * Using `main` for the main site makes sense to me because there's a lot of content that isn't specific to a version, like the splash page, spec page, community page, etc. I think those should live in one consistent place, which is probably under `main`. -- 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: dev-unsubscr...@iceberg.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [iceberg-docs] jackye1995 commented on pull request #1: First version of hugo doc site
jackye1995 commented on pull request #1: URL: https://github.com/apache/iceberg-docs/pull/1#issuecomment-989319785 > I'd focus on getting just 0.12.1 done. We can add more versions later Yeah I was just gonna ask about that, I thought we are going to first create those doc branches for stable versions, and then handle the `next` version. So my current understanding is that: 1. this PR will handle all the pages that are not specific to version to get a basic website out 2. have 1 PR per historical branch to port version specific documentation, focus on 0.12.1 as the first PR and working backwards 3. After 0.13 releases, we will create one branch for that 4. start to setup the `next` version that sync from the Iceberg repo Is this flow still correct? -- 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: dev-unsubscr...@iceberg.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [iceberg-docs] rdblue commented on pull request #1: First version of hugo doc site
rdblue commented on pull request #1: URL: https://github.com/apache/iceberg-docs/pull/1#issuecomment-989323751 Yeah, that sounds reasonable to me. I still question whether we even want to have `next`. In any case, there's still a lot of work to do here to make sure we have everything the way we want it. -- 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: dev-unsubscr...@iceberg.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [iceberg-docs] jackye1995 commented on pull request #1: First version of hugo doc site
jackye1995 commented on pull request #1: URL: https://github.com/apache/iceberg-docs/pull/1#issuecomment-989324679 yes, `next` is really good-to-have stuff, I saw it in a few other Apache websites that's why I brought it up, but if it's too much effort for community to maintain then we can opt-out. Let's definitely focus on getting the historical stable versions out first. -- 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: dev-unsubscr...@iceberg.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [iceberg-docs] samredai opened a new pull request #2: Landing-Page Site and Versioned Docs Site
samredai opened a new pull request #2: URL: https://github.com/apache/iceberg-docs/pull/2 This addresses 3 main issues raised in #1. - Broken Links - All links have been fixed and verified with https://github.com/filiph/linkcheck - Split the landing page and the docs site with only the docs site versioned - The deployment workflow has two jobs, `deploy-landing-page` and `deploy-docs` - Commits to `main` deploys **only** the single landing page - Commits to any non-`main` branch deploys **only** a docs site version (explained in the README) - Too many items in the navbar - The navbar has been shortened by hidding some pages and making them available in consolidated single pages with a list of links. See the `About` and `ASF` sections. - This is completely flexible and so any pages can be exposed in any of the following 3 places 1. Landing Page 2. Left Navbar 3. Within a Page View - (Let me know if any item in one of these places feels like it should be in another place) I have all of this running in my fork at https://samredai.github.io/iceberg-docs To check for broken links I ran these commands: ``` docker run --rm tennox/linkcheck https://samredai.github.io/iceberg-docs > landing-page-linkcheck.txt ``` ``` docker run --rm tennox/linkcheck https://samredai.github.io/iceberg-docs/docs/0.12.1 > 0.12.1-docsite-linkcheck.txt ``` ``` docker run --rm tennox/linkcheck https://samredai.github.io/iceberg-docs/docs/0.12.0 > 0.12.0-docsite-linkcheck.txt ``` There are a number of broken links that show up but they're all links to the javadocs (cat command into `grep -v 'javadoc'` filters out all of the broken links detected). Once we add the javadocs I can address those. -- 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: dev-unsubscr...@iceberg.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [iceberg-docs] samredai commented on pull request #1: First version of hugo doc site
samredai commented on pull request #1: URL: https://github.com/apache/iceberg-docs/pull/1#issuecomment-992183368 Closing this and continuing in #2 -- 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: dev-unsubscr...@iceberg.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [iceberg-docs] samredai closed pull request #1: First version of hugo doc site
samredai closed pull request #1: URL: https://github.com/apache/iceberg-docs/pull/1 -- 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: dev-unsubscr...@iceberg.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [iceberg-docs] rdblue commented on pull request #2: Landing-Page Site and Versioned Docs Site
rdblue commented on pull request #2: URL: https://github.com/apache/iceberg-docs/pull/2#issuecomment-994162460 Here are a few notes from clicking around: * Is the final path going to be `iceberg.apache.org/iceberg-docs`? Why not make the landing page at the root level? * In the preview, the "Learn More" link is broken. It points to https://samredai.github.io/docs instead of https://samredai.github.io/iceberg-docs/docs/, which is where the docs link takes you. * Since we also have a new front page, what about having the Docs link send you to a Getting Started page? * Clicking on "Tables" in the docs left nav is an empty page. Maybe we can make that a set of links to learn more about general table topics? * Same with Integrations, API, and Format. The text is a link, but there's no page there. * Under releases, should we include "latest"? -- 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: dev-unsubscr...@iceberg.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [iceberg-docs] rdblue edited a comment on pull request #2: Landing-Page Site and Versioned Docs Site
rdblue edited a comment on pull request #2: URL: https://github.com/apache/iceberg-docs/pull/2#issuecomment-994162460 Here are a few notes from clicking around: * Is the final path going to be `iceberg.apache.org/iceberg-docs`? Why not make the landing page at the root level? * In the preview, the "Learn More" link is broken. It points to https://samredai.github.io/docs instead of https://samredai.github.io/iceberg-docs/docs/, which is where the docs link takes you. * Since we also have a new front page, what about having the Docs link send you to a Getting Started page? * Clicking on "Tables" in the docs left nav is an empty page. Maybe we can make that a set of links to learn more about general table topics? * Same with Integrations, API, and Format. The text is a link, but there's no page there. * Under releases, should we include "latest"? * The About page is odd and just links to other pages * Looks like the ASF and About nav pages are there because we've removed the subpages from the nav. Is it possible to make those collapsible? * It's hard to find the pages under About. Those pages are important, like Blogs, Talks, Community, Releases, etc. -- 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: dev-unsubscr...@iceberg.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [iceberg-docs] rdblue edited a comment on pull request #2: Landing-Page Site and Versioned Docs Site
rdblue edited a comment on pull request #2: URL: https://github.com/apache/iceberg-docs/pull/2#issuecomment-994162460 Here are a few notes from clicking around: * Is the final path going to be `iceberg.apache.org/iceberg-docs`? Why not make the landing page at the root level? * In the preview, the "Learn More" link is broken. It points to https://samredai.github.io/docs instead of https://samredai.github.io/iceberg-docs/docs/, which is where the docs link takes you. * Since we also have a new front page, what about having the Docs link send you to a Getting Started page? * Clicking on "Tables" in the docs left nav is an empty page. Maybe we can make that a set of links to learn more about general table topics? * Same with Integrations, API, and Format. The text is a link, but there's no page there. * Under releases, should we include "latest"? * The About page is odd and just links to other pages * Looks like the ASF and About nav pages are there because we've removed the subpages from the nav. Is it possible to make those collapsible? * It's hard to find the pages under About. Those pages are important, like Blogs, Talks, Community, Releases, etc. * What does search do? When I search for something like `write.format.default`, there isn't much context, only a list of pages that look unrelated. The pages also show up in the nav and push everything down. -- 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: dev-unsubscr...@iceberg.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [iceberg-docs] rdblue edited a comment on pull request #2: Landing-Page Site and Versioned Docs Site
rdblue edited a comment on pull request #2: URL: https://github.com/apache/iceberg-docs/pull/2#issuecomment-994162460 Here are a few notes from clicking around: * Is the final path going to be `iceberg.apache.org/iceberg-docs`? Why not make the landing page at the root level? * In the preview, the "Learn More" link is broken. It points to https://samredai.github.io/docs instead of https://samredai.github.io/iceberg-docs/docs/, which is where the docs link takes you. * Since we also have a new front page, what about having the Docs link send you to a Getting Started page? * Clicking on "Tables" in the docs left nav is an empty page. Maybe we can make that a set of links to learn more about general table topics? * Same with Integrations, API, and Format. The text is a link, but there's no page there. * Under releases, should we include "latest"? * The About page is odd and just links to other pages * Looks like the ASF and About nav pages are there because we've removed the subpages from the nav. Is it possible to make those collapsible? * It's hard to find the pages under About. Those pages are important, like Blogs, Talks, Community, Releases, etc. * What does search do? When I search for something like `write.format.default`, there isn't much context, only a list of pages that look unrelated. The pages also show up in the nav and push everything down. * Clicking the Iceberg logo in the upper left in /docs brings you back to the docs splash page, not the Iceberg splash site. -- 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: dev-unsubscr...@iceberg.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [iceberg-docs] rdblue edited a comment on pull request #2: Landing-Page Site and Versioned Docs Site
rdblue edited a comment on pull request #2: URL: https://github.com/apache/iceberg-docs/pull/2#issuecomment-994162460 Here are a few notes from clicking around: * Is the final path going to be `iceberg.apache.org/iceberg-docs`? Why not make the landing page at the root level? * In the preview, the "Learn More" link is broken. It points to https://samredai.github.io/docs instead of https://samredai.github.io/iceberg-docs/docs/, which is where the docs link takes you. * Since we also have a new front page, what about having the Docs link send you to a Getting Started page? * Clicking on "Tables" in the docs left nav is an empty page. Maybe we can make that a set of links to learn more about general table topics? * Same with Integrations, API, and Format. The text is a link, but there's no page there. * Under releases, should we include "latest"? * The About page is odd and just links to other pages * Looks like the ASF and About nav pages are there because we've removed the subpages from the nav. Is it possible to make those collapsible? * It's hard to find the pages under About. Those pages are important, like Blogs, Talks, Community, Releases, etc. * What does search do? When I search for something like `write.format.default`, there isn't much context, only a list of pages that look unrelated. The pages also show up in the nav and push everything down. * Clicking the Iceberg logo in the upper left in /docs brings you back to the docs splash page, not the Iceberg splash site. * The splash site uses "Mailing lists" instead of "Community". Could you link to the community page instead? -- 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: dev-unsubscr...@iceberg.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [iceberg-docs] rdblue edited a comment on pull request #2: Landing-Page Site and Versioned Docs Site
rdblue edited a comment on pull request #2: URL: https://github.com/apache/iceberg-docs/pull/2#issuecomment-994162460 Here are a few notes from clicking around: * Is the final path going to be `iceberg.apache.org/iceberg-docs`? Why not make the landing page at the root level? * In the preview, the "Learn More" link is broken. It points to https://samredai.github.io/docs instead of https://samredai.github.io/iceberg-docs/docs/, which is where the docs link takes you. * Since we also have a new front page, what about having the Docs link send you to a Getting Started page? * Clicking on "Tables" in the docs left nav is an empty page. Maybe we can make that a set of links to learn more about general table topics? * Same with Integrations, API, and Format. The text is a link, but there's no page there. * Under releases, should we include "latest"? * The About page is odd and just links to other pages * Looks like the ASF and About nav pages are there because we've removed the subpages from the nav. Is it possible to make those collapsible? * It's hard to find the pages under About. Those pages are important, like Blogs, Talks, Community, Releases, etc. * What does search do? When I search for something like `write.format.default`, there isn't much context, only a list of pages that look unrelated. The pages also show up in the nav and push everything down. * Clicking the Iceberg logo in the upper left in /docs brings you back to the docs splash page, not the Iceberg splash site. * The splash site uses "Mailing lists" instead of "Community". Could you link to the community page instead? * On the splash page, I think we should add links to the features that are highlighted. We should also consider what to put here. The 3 listed don't seem compelling enough. -- 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: dev-unsubscr...@iceberg.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [iceberg-docs] jackye1995 commented on pull request #2: Landing-Page Site and Versioned Docs Site
jackye1995 commented on pull request #2: URL: https://github.com/apache/iceberg-docs/pull/2#issuecomment-994422440 Also tried the website, thanks for continuing the work! It seems like some links are still broken as Ryan points out. I have some additional nitpicking: 1. Github and Slack buttons should open in a new page instead of loading in the current page. 2. the time travel script is setting a table property to perform time-travel, can we use the latest time travel syntax `AS OF`? 3. for the issue that there is no link for those top level tabs like "Integration", I think we can just disable the link? Because the 2 that has link (Spark and Flink) are all linking to the getting started page, I think we can make them just a new sub-tab called `Getting started` and disable all top level tab links. 4. Once I am in the doc page, I cannot go back to the landing page anymore, can we at least have a way to go back? 5. also agree that feature list is not full, we can think about the list of features together -- 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: dev-unsubscr...@iceberg.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [iceberg-docs] samredai commented on pull request #2: Landing-Page Site and Versioned Docs Site
samredai commented on pull request #2: URL: https://github.com/apache/iceberg-docs/pull/2#issuecomment-997086542 Thanks for all the feedback! Here's an updated demo site to show all the fixes: https://iceberg.redai.dev/ @rdblue Is the final path going to be iceberg.apache.org/iceberg-docs? Why not make the landing page at the root level? - No there won't be any "/iceberg-docs". This is mainly because I'm using the stock personal gh-pages without a custom domain so it appends the repo name. I had a domain lying around that just plugged into this demo (once it's merged we'll be able to do "dev" deployments of the site to something like "/unstable"). So now the demo urls will match how the site will look, besides the domain of course. (see https://iceberg.redai.dev/) In the preview, the "Learn More" link is broken. It points to https://samredai.github.io/docs instead of https://samredai.github.io/iceberg-docs/docs/, which is where the docs link takes you. - This is slightly related to the previous item and is also fixed. Since we also have a new front page, what about having the Docs link send you to a Getting Started page? - I like it! I changed Learn More to link to the spark getting-started page (we can update this as we add more getting started material) Clicking on "Tables" in the docs left nav is an empty page. Maybe we can make that a set of links to learn more about general table topics? - Removed the empty link, and that's a great idea. When we add the new set of getting started material I'll re-link this section titles. Same with Integrations, API, and Format. The text is a link, but there's no page there. - Fixed by removing the link Under releases, should we include "latest"? - I was overthinking this a bit and removed latest because it's a clone of the 0.12.1 site. Thinking about it now it actually makes sense to keep them as two different sites so I updated this. Also included a link to the "Release Notes" page as an item instead of linking the section title which wasn't obvious. The About page is odd and just links to other pages - Fixed when changing the left nav to collapsible sections at the bottom Looks like the ASF and About nav pages are there because we've removed the subpages from the nav. Is it possible to make those collapsible? - Fixed, these are now collapsible It's hard to find the pages under About. Those pages are important, like Blogs, Talks, Community, Releases, etc. - These are now included in the left nav What does search do? When I search for something like write.format.default, there isn't much context, only a list of pages that look unrelated. The pages also show up in the nav and push everything down. - Wow...thanks for pointing this out! I looked into this and it turns out this theme is using [flexsearch](https://github.com/nextapps-de/flexsearch) which is a pretty advanced full-text search library with some scoring algorithm that's doing way too much to predict relevance (and clearly missing the mark). Hugo already provides a way to generate a search index using the templates functionality (see `search-data.json`). I replaced all of the flexsearch code with a simple lookup of the search box's value to the content and a few lines of javascript to show a preview of where the match is and to link to the page. It looks to be working well and it lets us remove the flexsearch dependency. Clicking the Iceberg logo in the upper left in /docs brings you back to the docs splash page, not the Iceberg splash site. - Fixed! The splash site uses "Mailing lists" instead of "Community". Could you link to the community page instead? - Fixed! Github and Slack buttons should open in a new page instead of loading in the current page. - Fixed! On the splash page, I think we should add links to the features that are highlighted. We should also consider what to put here. The 3 listed don't seem compelling enough. - Great idea, I'll add this as I work on a more compelling feature set to display @jackye1995 for the issue that there is no link for those top level tabs like "Integration", I think we can just disable the link? Because the 2 that has link (Spark and Flink) are all linking to the getting started page, I think we can make them just a new sub-tab called Getting started and disable all top level tab links. - Done! Collapsing the lower sections freed up a lot of space to add this in it's on line. Once I am in the doc page, I cannot go back to the landing page anymore, can we at least have a way to go back? - Fixed this by changing the Iceberg logo to return to the splash page like @rdblue suggested the time travel script is setting a table property to perform time-travel, can we use the latest time travel syntax AS OF? - Will do!
[GitHub] [iceberg-docs] samredai edited a comment on pull request #2: Landing-Page Site and Versioned Docs Site
samredai edited a comment on pull request #2: URL: https://github.com/apache/iceberg-docs/pull/2#issuecomment-997086542 Thanks for all the feedback! Here's an updated demo site to show all the fixes: https://iceberg.redai.dev/ @rdblue Is the final path going to be iceberg.apache.org/iceberg-docs? Why not make the landing page at the root level? - No there won't be any "/iceberg-docs". This is mainly because I'm using the stock personal gh-pages without a custom domain so it appends the repo name. I had a domain lying around that just plugged into this demo (once it's merged we'll be able to do "dev" deployments of the site to something like "/unstable"). So now the demo urls will match how the site will look, besides the domain of course. (see https://iceberg.redai.dev/) In the preview, the "Learn More" link is broken. It points to https://samredai.github.io/docs instead of https://samredai.github.io/iceberg-docs/docs/, which is where the docs link takes you. - This is slightly related to the previous item and is also fixed. Since we also have a new front page, what about having the Docs link send you to a Getting Started page? - I like it! I changed Learn More to link to the spark getting-started page (we can update this as we add more getting started material) Clicking on "Tables" in the docs left nav is an empty page. Maybe we can make that a set of links to learn more about general table topics? - Removed the empty link, and that's a great idea. When we add the new set of getting started material I'll re-link this section titles. Same with Integrations, API, and Format. The text is a link, but there's no page there. - Fixed by removing the link Under releases, should we include "latest"? - I was overthinking this a bit and removed latest because it's a clone of the 0.12.1 site. Thinking about it now it actually makes sense to keep them as two different sites so I updated this. Also included a link to the "Release Notes" page as an item instead of linking the section title which wasn't obvious. The About page is odd and just links to other pages - Fixed when changing the left nav to collapsible sections at the bottom Looks like the ASF and About nav pages are there because we've removed the subpages from the nav. Is it possible to make those collapsible? - Fixed, these are now collapsible It's hard to find the pages under About. Those pages are important, like Blogs, Talks, Community, Releases, etc. - These are now included in the left nav What does search do? When I search for something like write.format.default, there isn't much context, only a list of pages that look unrelated. The pages also show up in the nav and push everything down. - Wow...thanks for pointing this out! I looked into this and it turns out this theme is using [flexsearch](https://github.com/nextapps-de/flexsearch) which is a pretty advanced full-text search library with some scoring algorithm that's doing way too much to predict relevance (and clearly missing the mark). Hugo already provides a way to generate a search index using the templates functionality (see `search-data.json`). I replaced all of the flexsearch code with a simple lookup of the search box's value to the content and a few lines of javascript to show a preview of where the match is and to link to the page. It looks to be working well and it lets us remove the flexsearch dependency. Clicking the Iceberg logo in the upper left in /docs brings you back to the docs splash page, not the Iceberg splash site. - Fixed! The splash site uses "Mailing lists" instead of "Community". Could you link to the community page instead? - Fixed! On the splash page, I think we should add links to the features that are highlighted. We should also consider what to put here. The 3 listed don't seem compelling enough. - Great idea, I'll add this as I work on a more compelling feature set to display @jackye1995 Github and Slack buttons should open in a new page instead of loading in the current page. - Fixed! for the issue that there is no link for those top level tabs like "Integration", I think we can just disable the link? Because the 2 that has link (Spark and Flink) are all linking to the getting started page, I think we can make them just a new sub-tab called Getting started and disable all top level tab links. - Done! Collapsing the lower sections freed up a lot of space to add this in it's on line. Once I am in the doc page, I cannot go back to the landing page anymore, can we at least have a way to go back? - Fixed this by changing the Iceberg logo to return to the splash page like @rdblue suggested the time travel script is setting a table property to perform time-travel, can we use the latest time travel syntax AS OF? -
[GitHub] [iceberg-docs] rdblue commented on pull request #2: Landing-Page Site and Versioned Docs Site
rdblue commented on pull request #2: URL: https://github.com/apache/iceberg-docs/pull/2#issuecomment-997489501 Thanks, @samredai! The site looks much better. Before, we talked about having some pages that are not versioned and have just one copy throughout. Things like the Releases page (since that states the current release and is stale in 0.12.0) and the Spec would benefit from this. Is this possible? Otherwise, I think the only remaining thing to fix is the splash page. Maybe we should merge this and then work on those two issues? @jackye1995 what do you think? Are we ready to move forward with this since it's in a separate repo and make improvements in parallel PRs? -- 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: dev-unsubscr...@iceberg.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [iceberg-docs] samredai commented on pull request #2: Landing-Page Site and Versioned Docs Site
samredai commented on pull request #2: URL: https://github.com/apache/iceberg-docs/pull/2#issuecomment-998133549 Done! Here's what I came up with: - Moved all of the common pages to `docs/commons`. This includes the Community, Releases, and Format sections but let me know if there's any other pages (we can get as granular as a single markdown file). - Added a [copy_to_docs.sh](https://github.com/apache/iceberg-docs/blob/c657a5566c5496178efe573d4e94225cb4d80b0d/docs/common/copy_to_docs.sh) script that copies everything in `docs/common` into `docs/content`. - Added a step in the [deployment workflow](https://github.com/apache/iceberg-docs/blob/c657a5566c5496178efe573d4e94225cb4d80b0d/.github/workflows/deploy.yml#L85) to run the `copy_to_docs.sh` script during the `deploy-docs` job. @rdblue @jackye1995 let me know what you think of this setup and I can add a section to the README explaining this. -- 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: dev-unsubscr...@iceberg.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [iceberg-docs] jackye1995 commented on pull request #2: Landing-Page Site and Versioned Docs Site
jackye1995 commented on pull request #2: URL: https://github.com/apache/iceberg-docs/pull/2#issuecomment-998987824 @rdblue yes I totally agree we can move forward and make incremental progress on this, that also makes the review much easier with more specific changes to look at. -- 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: dev-unsubscr...@iceberg.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [iceberg-docs] kbendick commented on a change in pull request #2: Landing-Page Site and Versioned Docs Site
kbendick commented on a change in pull request #2: URL: https://github.com/apache/iceberg-docs/pull/2#discussion_r773448184 ## File path: .github/workflows/deploy.yml ## @@ -0,0 +1,98 @@ +name: github pages + +on: [push, pull_request] + +jobs: + deploy-landing-page: +if: github.ref == 'refs/heads/main' +runs-on: ubuntu-20.04 +steps: + - uses: actions/checkout@v2 + Review comment: Question: Is this `uses` supposed to be part of the same block as `- name: Set output`? -- 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: dev-unsubscr...@iceberg.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [iceberg-docs] rdblue commented on pull request #2: Landing-Page Site and Versioned Docs Site
rdblue commented on pull request #2: URL: https://github.com/apache/iceberg-docs/pull/2#issuecomment-999187098 @samredai, if I understand the workflow correctly, it sounds like you're making extra copies of the common pages, one in each set of docs? I think that still has a problem because there isn't a trigger that will rebuild the old docs branches when there is an update to the main content right? What about just making those pages part of the splash site at the top level and linking to them from all the docs branches? -- 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: dev-unsubscr...@iceberg.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [iceberg-docs] samredai commented on pull request #2: Landing-Page Site and Versioned Docs Site
samredai commented on pull request #2: URL: https://github.com/apache/iceberg-docs/pull/2#issuecomment-999728798 @rdblue Yes you're understanding is correct. That's a good point, updating the common pages would still require redeploying the previous sites (even though the same document is deployed). Let me move this over to the landing-page site. It'll require a little extra designing since it will be out of the docs site and the landing-page site doesn't have any single page templates. I'll put together a simple page template that matches the font and color of the main site, shows the markdown content, and has a simple set of links to either the home page (landing-page) or the docs site. That would let us do what you described, keep it as part of the splash site build and link to it from the docs branches! -- 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: dev-unsubscr...@iceberg.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [iceberg-docs] samredai opened a new pull request #3: Add .asf.yaml file with notification settings
samredai opened a new pull request #3: URL: https://github.com/apache/iceberg-docs/pull/3 This adds an .asf.yaml file with notification settings to send commit notifications to comm...@iceberg.apache.org and issue/pr notifications to iss...@iceberg.apache.org. -- 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: dev-unsubscr...@iceberg.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [iceberg-docs] rdblue commented on a change in pull request #3: Add .asf.yaml file with notification settings
rdblue commented on a change in pull request #3: URL: https://github.com/apache/iceberg-docs/pull/3#discussion_r777680979 ## File path: .asf.yaml ## @@ -0,0 +1,34 @@ +# +# 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. +# + +# The format of this file is documented at +# https://cwiki.apache.org/confluence/display/INFRA/Git+-+.asf.yaml+features + +github: + description: "Apache Icebergi Documentation Site" + homepage: https://iceberg.apache.org/ + labels: +- iceberg +- apache Review comment: docs? -- 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: dev-unsubscr...@iceberg.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [iceberg-docs] rdblue commented on a change in pull request #3: Add .asf.yaml file with notification settings
rdblue commented on a change in pull request #3: URL: https://github.com/apache/iceberg-docs/pull/3#discussion_r777681084 ## File path: .asf.yaml ## @@ -0,0 +1,34 @@ +# +# 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. +# + +# The format of this file is documented at +# https://cwiki.apache.org/confluence/display/INFRA/Git+-+.asf.yaml+features + +github: + description: "Apache Icebergi Documentation Site" Review comment: Typo: Icebergi -- 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: dev-unsubscr...@iceberg.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [iceberg-docs] samredai commented on a change in pull request #3: Add .asf.yaml file with notification settings
samredai commented on a change in pull request #3: URL: https://github.com/apache/iceberg-docs/pull/3#discussion_r17386 ## File path: .asf.yaml ## @@ -0,0 +1,34 @@ +# +# 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. +# + +# The format of this file is documented at +# https://cwiki.apache.org/confluence/display/INFRA/Git+-+.asf.yaml+features + +github: + description: "Apache Icebergi Documentation Site" + homepage: https://iceberg.apache.org/ + labels: +- iceberg +- apache Review comment: Added -- 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: dev-unsubscr...@iceberg.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [iceberg-docs] samredai commented on a change in pull request #3: Add .asf.yaml file with notification settings
samredai commented on a change in pull request #3: URL: https://github.com/apache/iceberg-docs/pull/3#discussion_r17576 ## File path: .asf.yaml ## @@ -0,0 +1,34 @@ +# +# 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. +# + +# The format of this file is documented at +# https://cwiki.apache.org/confluence/display/INFRA/Git+-+.asf.yaml+features + +github: + description: "Apache Icebergi Documentation Site" Review comment: Fixed, thanks! -- 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: dev-unsubscr...@iceberg.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [iceberg-docs] rdblue commented on pull request #3: Add .asf.yaml file with notification settings
rdblue commented on pull request #3: URL: https://github.com/apache/iceberg-docs/pull/3#issuecomment-1004398533 I merged this using the CLI since I didn't see that I could squash and merge in the dropdown. We'll have to figure out how to remove merge commits as an option. Thanks, @samredai! -- 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: dev-unsubscr...@iceberg.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[GitHub] [iceberg-docs] samredai closed pull request #3: Add .asf.yaml file with notification settings
samredai closed pull request #3: URL: https://github.com/apache/iceberg-docs/pull/3 -- 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: dev-unsubscr...@iceberg.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org