jackye1995 commented on a change in pull request #2403:
URL: https://github.com/apache/iceberg/pull/2403#discussion_r606469713



##########
File path: core/src/main/java/org/apache/iceberg/SerializedTable.java
##########
@@ -0,0 +1,312 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg;
+
+import java.io.Serializable;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.encryption.EncryptionManager;
+import org.apache.iceberg.hadoop.HadoopFileIO;
+import org.apache.iceberg.hadoop.SerializedConfiguration;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.LocationProvider;
+import org.apache.iceberg.util.SerializableMap;
+
+/**
+ * A read-only serializable table implementation that can be sent to other 
nodes.
+ * <p>
+ * While this class captures the metadata file location that can be used to 
load the full table
+ * metadata, it also serializes directly things like properties, schema, spec, 
sort order
+ * to avoid reading the metadata file from other nodes to access frequently 
needed metadata.
+ * <p>
+ * <em>Note:</em> loading the full metadata from a large number of nodes can 
overwhelm the storage.
+ */
+public class SerializedTable implements Table, Serializable {
+  private final String name;
+  private final String location;
+  private final String metadataFileLocation;
+  private final Map<String, String> properties;
+  private final String schemaAsJson;
+  private final String specAsJson;
+  private final String sortOrderAsJson;
+  private final FileIO io;
+  private final EncryptionManager encryption;
+  private final LocationProvider locationProvider;
+
+  private transient volatile Table lazyTable = null;
+  private transient volatile Schema lazySchema = null;
+  private transient volatile PartitionSpec lazySpec = null;
+  private transient volatile SortOrder lazySortOrder = null;
+
+  public SerializedTable(Table table) {
+    this.name = table.name();
+    this.location = table.location();
+    this.metadataFileLocation = metadataFileLocation(table);
+    this.properties = SerializableMap.copyOf(table.properties());
+    this.schemaAsJson = SchemaParser.toJson(table.schema());
+    this.specAsJson = PartitionSpecParser.toJson(table.spec());
+    this.sortOrderAsJson = SortOrderParser.toJson(table.sortOrder());
+    this.io = fileIO(table);
+    this.encryption = table.encryption();
+    this.locationProvider = table.locationProvider();
+  }
+
+  private String metadataFileLocation(Table table) {
+    if (table instanceof HasTableOperations) {
+      TableOperations ops = ((HasTableOperations) table).operations();
+      return ops.current().metadataFileLocation();
+    } else {
+      return null;
+    }
+  }
+
+  private FileIO fileIO(Table table) {

Review comment:
       Cool, let's discuss in another PR then. Just to throw another idea I 
just have:
   
   ```java
   public interface KryoSerializable<T> extends Serializable {
     
     // return a serialized version of self
     default T serialized() {
       throw new UnsupportedOperationException("Cannot support kryo 
serialization");
     }
   }
   ```
   
   ```java
   public interface FileIO extends KryoSerializable<FileIO> {
    ...
   }
   ```
   
   ```java
   public class HadoopFileIO ... {
     @Override
     public FileIO serialized() {
       SerializedConfiguration serializedConf = new 
SerializedConfiguration(getConf());
       return new HadoopFileIO(serializedConf::get);
     }
   }
   ```
   
   By doing this, we enforce everything dynamically loaded to be kryo 
serializable.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
[email protected]



---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to