Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Support customizing the location where data is written in Spark #6

Merged
merged 13 commits into from
Dec 11, 2018

Conversation

mccheah
Copy link
Contributor

@mccheah mccheah commented Nov 26, 2018

Closes Netflix/iceberg#93.

Note that we only use this in the Spark writer, but this also has to be worked into the other integrations.

.orElse(table.properties().getOrDefault(
TableProperties.WRITE_NEW_DATA_LOCATION,
new Path(new Path(table.location()), "data").toString()));
return Optional.of(new Writer(table, lazyConf(), format, dataLocation));
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Instead of adding parameters to Writer whenever a change like this is made, I'd rather pass the options into Writer and handle these there. The dataLocation method could do this work instead of moving it outside the Writer class.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think doing options processing from a Map<String, String>, inside a constructor, is a bit of an antipattern. Consider for example writing a unit test for this class in the future. If we pass the Writer constructor only a HashMap, the unit test would have to construct that HashMap in a specific way, i.e. knowing what key-value pairs the constructor is expecting.

Perhaps we can have a builder object that acts as a factory that accepts the Map and returns the Writer. The Writer constructor accepts the builder object and copies the set fields on the builder into its own fields.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

What I think is strange is passing the location of a write into the writer when we're passing table into the writer. Why isn't that logic entirely handled in the writer? The normal case is for the write location to come from table config. I'm not even sure that we should allow overriding the write location in Spark's write properties. What is the use case there?

I like your reasoning about not passing options as a map to make testing clear in general, but doing it here just shifts the concern to a different test. The test case is that setting "write.folder-storage.path" in Spark options changes the location of output files. A test that passes in the location can validate that the location is respected, but what we actually want to do is test that the table's location defaults, or is set by the table property, or (maybe) is set by Spark options.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think for our use case we can have the write location specified in the table property. That would be sufficient. I also don't see the downside of introducing the extra flexibility of allowing the override to be specified in data source options, but we could defer the feature until later.

import com.netflix.iceberg.hadoop.HadoopTables;
import com.netflix.iceberg.spark.SparkSchemaUtil;
import com.netflix.iceberg.types.CheckCompatibility;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Nit: imports should be a single block, not separated by newlines.


// This only applies to files written after this property is set. Files previously written aren't relocated to
// reflect this parameter.
public static final String WRITE_NEW_DATA_LOCATION = "write.data.location";
Copy link
Contributor

@rdblue rdblue Nov 26, 2018

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Should this property mirror the existing write.object-storage.path property? Maybe write.folder-storage.path would be better.

Also, I would like to see a comment about what happens when this isn't set. That behavior should be to default to a data folder under the table location. These properties should be respected by all writers, so we want to have them well documented.

@@ -71,7 +72,9 @@ public static void stopSpark() {
public void testBasicWrite() throws IOException {
File parent = temp.newFolder("parquet");
File location = new File(parent, "test");
File dataLocation = new File(parent, "test-data");
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This test should not be modified because it tests the default behavior. There should be an additional test for behavior with the new property. I'd also like to see a test that sets the property on the table instead of in the write options, and another one where both are set and the write option takes precedence.

Also, I think this should probably go into a more general test suite. This one is specific to Parquet, but you're testing the behavior for any file format.

@Override
public String toString() {
return String.format("IcebergWrite(table=%s, type=%s, format=%s)",
table, table.schema().asStruct(), format);
}


Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Nit: whitespace-only change.


private void writeAndValidateWithLocations(
Schema schema,
boolean setTablePropertyDataLocation,
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Hm, rereading this I think it tries too hard to reuse code in exchange for the antipattern of using boolean switches. This can be written more idiomatically.

@mccheah
Copy link
Contributor Author

mccheah commented Dec 11, 2018

Addressed the comments and is ready for another round of review. Also made the test cleaner.


@Parameterized.Parameters
public static Object[][] parameters() {
return new Object[][] {
new Object[] { "parquet" },
new Object[] { "parquet" },
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Why did you add these?

@rdblue
Copy link
Contributor

rdblue commented Dec 11, 2018

Looks good to me, other than the duplicate test cases in parameters.

@rdblue rdblue merged commit 0342f23 into apache:master Dec 11, 2018
@rdblue
Copy link
Contributor

rdblue commented Dec 11, 2018

Merged. Thanks @mccheah!

rdblue referenced this pull request in rdblue/iceberg Dec 21, 2018
This adds a new table property, write.folder-storage.path, that controls the location of new data files.
yifeih pushed a commit to yifeih/incubator-iceberg that referenced this pull request Apr 16, 2019
* Publish to Bintray

* Upgrade shadow plugin

* Make bintray name equal to repository name
prodeezy pushed a commit to rominparekh/incubator-iceberg that referenced this pull request Dec 17, 2019
# This is the 1st commit message:

Issue-629: Cherrypick Id

# This is the commit message #2:

Removed redundant methods and changed method name

# This is the commit message #3:

Fix Imports

# This is the commit message #4:

Fix Operation Check

# This is the commit message apache#5:

Fix Error Message

# This is the commit message apache#6:

Cherry picking operation to apply changes from incoming snapshot on current snapshot

# This is the commit message apache#7:

Initial working version of cherry-pick operation which applies appends only
guilload pushed a commit to guilload/iceberg that referenced this pull request Jul 9, 2020
* Fix mapred serialization bug
moulimukherjee referenced this pull request in moulimukherjee/iceberg Jul 24, 2020
puchengy pushed a commit to puchengy/iceberg that referenced this pull request Apr 17, 2022
…el using catalog properties. (apache#6)

* Allow table defaults to be configured and/ or enforced at catalog level using catalog properties.

* Make catalogProps field private

* Updates

* Minor cleanup

* Update
adamyasharma2797 pushed a commit to adamyasharma2797/iceberg that referenced this pull request Jul 19, 2024
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
Development

Successfully merging this pull request may close these issues.

2 participants