This is an automated email from the ASF dual-hosted git repository.

sivabalan pushed a commit to branch asf-site
in repository https://gitbox.apache.org/repos/asf/hudi.git


The following commit(s) were added to refs/heads/asf-site by this push:
     new af6d23d  [HUDI-2543]: Added guides section (#3776)
af6d23d is described below

commit af6d23d45d8d5e325455b4f95fada676c03c21e2
Author: Pratyaksh Sharma <[email protected]>
AuthorDate: Thu Dec 16 01:35:19 2021 +0530

    [HUDI-2543]: Added guides section (#3776)
---
 website/docs/troubleshooting.md | 107 +++++++++++++++++++++++++++++++++++++++-
 website/docs/tuning-guide.md    |  57 +++++++++++++++++++++
 website/sidebars.js             |   1 +
 3 files changed, 164 insertions(+), 1 deletion(-)

diff --git a/website/docs/troubleshooting.md b/website/docs/troubleshooting.md
index 39f5d34..b391813 100644
--- a/website/docs/troubleshooting.md
+++ b/website/docs/troubleshooting.md
@@ -13,7 +13,7 @@ Section below generally aids in debugging Hudi failures. Off 
the bat, the follow
 - **_hoodie_file_name** - Actual file name containing the record (super useful 
to triage duplicates)
 - **_hoodie_partition_path** - Path from basePath that identifies the 
partition containing this record
 
-For performance related issues, please refer to the [tuning 
guide](https://cwiki.apache.org/confluence/display/HUDI/Tuning+Guide)
+For performance related issues, please refer to the [tuning 
guide](https://hudi.apache.org/docs/tuning-guide)
 
 
 ### Missing records
@@ -54,3 +54,108 @@ At a high level, there are two steps
 
 Depending on the exception source (Hudi/Spark), the above knowledge of the DAG 
can be used to pinpoint the actual issue. The most often encountered failures 
result from YARN/DFS temporary failures.
 In the future, a more sophisticated debug/management UI would be added to the 
project, that can help automate some of this debugging.
+
+### Common Issues
+
+This section lists down all the common issues that users have faced while 
using Hudi. [Contributions](https://hudi.apache.org/contribute/get-involved) 
are always welcome to improve this section.
+
+#### Writing Data
+
+##### Caused by: org.apache.parquet.io.InvalidRecordException: Parquet/Avro 
schema mismatch: Avro field 'col1' not found
+
+It is recommended that schema should evolve in [backwards compatible 
way](https://docs.confluent.io/platform/current/schema-registry/avro.html) 
while using Hudi. Please refer here for more information on avro schema 
resolution - https://avro.apache.org/docs/1.8.2/spec.html. This error generally 
occurs when the schema has evolved in backwards **incompatible** way by 
deleting some column 'col1' and we are trying to update some record in parquet 
file which has alredy been written with previ [...]
+
+The fix for this is to try and create uber schema using all the schema 
versions evolved so far for the concerned event and use this uber schema as the 
target schema. One of the good approaches can be fetching schema from hive 
metastore and merging it with the current schema.
+
+Sample stacktrace where a field named "toBeDeletedStr" was omitted from new 
batch of updates : 
https://gist.github.com/nsivabalan/cafc53fc9a8681923e4e2fa4eb2133fe
+
+##### Caused by: java.lang.UnsupportedOperationException: 
org.apache.parquet.avro.AvroConverters$FieldIntegerConverter
+
+This error will again occur due to schema evolutions in non-backwards 
compatible way. Basically there is some incoming update U for a record R which 
is already written to your Hudi dataset in the concerned parquet file. R 
contains field F which is having certain data type, let us say long. U has the 
same field F with updated data type of int type. Such incompatible data type 
conversions are not supported by Parquet FS.
+
+For such errors, please try to ensure only valid data type conversions are 
happening in your primary data source from where you are trying to ingest.
+
+Sample stacktrace when trying to evolve a field from Long type to Integer type 
with Hudi : https://gist.github.com/nsivabalan/0d81cd60a3e7a0501e6a0cb50bfaacea
+
+##### org.apache.hudi.exception.SchemaCompatabilityException: Unable to 
validate the rewritten record &lt;record&gt; against schema &lt;schema&gt; at 
org.apache.hudi.common.util.HoodieAvroUtils.rewrite(HoodieAvroUtils.java:215)
+
+This can possibly occur if your schema has some non-nullable field whose value 
is not present or is null. It is recommended to evolve schema in backwards 
compatible ways. In essence, this means either have every newly added field as 
nullable or define default values for every new field. In case if you are 
relying on default value for your field, as of Hudi version 0.5.1, this is not 
handled.
+
+##### hudi consumes too much space in a temp folder while upsert
+
+When upsert large input data, hudi will spills part of input data to disk when 
reach the max memory for merge. if there is enough memory, please increase 
spark executor's memory and  "hoodie.memory.merge.fraction" option, for example
+
+```java
+option("hoodie.memory.merge.fraction", "0.8")
+```
+
+#### Ingestion
+
+##### Caused by: java.io.EOFException: Received -1 when reading from channel, 
socket has likely been closed. at kafka.utils.Utils$.read(Utils.scala:381) at 
kafka.network.BoundedByteBufferReceive.readFrom(BoundedByteBufferReceive.scala:54)
+
+This might happen if you are ingesting from Kafka source, your cluster is ssl 
enabled by default and you are using some version of Hudi older than 0.5.1. 
Previous versions of Hudi were using spark-streaming-kafka-0-8 library. With 
the release of 0.5.1 version of Hudi, spark was upgraded to 2.4.4 and 
spark-streaming-kafka library was upgraded to spark-streaming-kafka-0-10. SSL 
support was introduced from spark-streaming-kafka-0-10. Please see here for 
reference.
+
+The workaround can be either use Kafka cluster which is not ssl enabled, else 
upgrade Hudi version to at least 0.5.1 or spark-streaming-kafka library to 
spark-streaming-kafka-0-10.
+
+##### Exception in thread "main" org.apache.kafka.common.KafkaException: 
Failed to construct kafka consumer
+##### Caused by: java.lang.IllegalArgumentException: Could not find a 
'KafkaClient' entry in the JAAS configuration. System property 
'java.security.auth.login.config' is not set
+
+This might happen when you are trying to ingest from ssl enabled kafka source 
and your setup is not able to read jars.conf file and its properties. To fix 
this, you need to pass the required property as part of your spark-submit 
command something like
+
+```java
+--files jaas.conf,failed_tables.json --conf 
'spark.driver.extraJavaOptions=-Djava.security.auth.login.config=jaas.conf' 
--conf 
'spark.executor.extraJavaOptions=-Djava.security.auth.login.config=jaas.conf'
+```
+
+##### com.uber.hoodie.exception.HoodieException: created_at(Part -created_at) 
field not found in record. Acceptable fields were :[col1, col2, col3, id, name, 
dob, created_at, updated_at]
+
+Happens generally when field marked as recordKey or partitionKey is not 
present in some incoming record. Please cross verify your incoming record once.
+
+##### If it is possible to use a nullable field that contains null records as 
a primary key when creating hudi table
+
+No, will throw HoodieKeyException
+
+```scala
+Caused by: org.apache.hudi.exception.HoodieKeyException: recordKey value: 
"null" for field: "name" cannot be null or empty.
+  at 
org.apache.hudi.keygen.SimpleKeyGenerator.getKey(SimpleKeyGenerator.java:58)
+  at 
org.apache.hudi.HoodieSparkSqlWriter$$anonfun$1.apply(HoodieSparkSqlWriter.scala:104)
+  at 
org.apache.hudi.HoodieSparkSqlWriter$$anonfun$1.apply(HoodieSparkSqlWriter.scala:100)
+```
+
+#### Hive Sync
+
+##### Caused by: java.sql.SQLException: Error while processing statement: 
FAILED: Execution Error, return code 1 from 
org.apache.hadoop.hive.ql.exec.DDLTask. Unable to alter table. The following 
columns have types incompatible with the existing columns in their respective 
positions : __col1,__col2
+
+This will usually happen when you are trying to add a new column to existing 
hive table using our HiveSyncTool.java class. Databases usually will not allow 
to modify a column datatype from a higher order to lower order or cases where 
the datatypes may clash with the data that is already stored/will be stored in 
the table. To fix the same, try setting the following property -
+
+```scala
+set hive.metastore.disallow.incompatible.col.type.changes=false;
+```
+
+##### com.uber.hoodie.hive.HoodieHiveSyncException: Could not convert field 
Type from &lt;type1&gt; to &lt;type2&gt; for field col1
+
+This occurs because HiveSyncTool currently supports only few compatible data 
type conversions. Doing any other incompatible change will throw this 
exception. Please check the data type evolution for the concerned field and 
verify if it indeed can be considered as a valid data type conversion as per 
Hudi code base.
+
+##### Caused by: org.apache.hadoop.hive.ql.parse.SemanticException: Database 
does not exist: test_db
+
+This generally occurs if you are trying to do Hive sync for your Hudi dataset 
and the configured hive_sync database does not exist. Please create the 
corresponding database on your Hive cluster and try again.
+
+##### Caused by: org.apache.thrift.TApplicationException: Invalid method name: 
'get_table_req'
+
+This issue is caused by hive version conflicts, hudi built with hive-2.3.x 
version, so if still want hudi work with older hive version
+
+```scala
+Steps: (build with hive-2.1.0)
+1. git clone [email protected]:apache/incubator-hudi.git
+2. rm 
hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/hive/HoodieCombineHiveInputFormat.java
+3. mvn clean package -DskipTests -DskipITs -Dhive.version=2.1.0
+```
+
+##### Caused by : java.lang.UnsupportedOperationException: Table rename is not 
supported
+
+This issue could occur when syncing to hive. Possible reason is that, hive 
does not play well if your table name has upper and lower case letter. Try to 
have all lower case letters for your table name and it should likely get fixed. 
Related issue: https://github.com/apache/hudi/issues/2409
+
+#### Running from IDE
+
+##### "java.lang.IllegalArgumentException: Unsupported class file major 
version 56"
+
+Please use java 8, and not java 11
diff --git a/website/docs/tuning-guide.md b/website/docs/tuning-guide.md
new file mode 100644
index 0000000..581778a
--- /dev/null
+++ b/website/docs/tuning-guide.md
@@ -0,0 +1,57 @@
+---
+title: Tuning Guide
+keywords: [hudi, tuning, performance]
+last_modified_at: 2021-09-29T15:59:57-04:00
+---
+# Tuning Guide
+
+:::note Profiling Tip
+To get a better understanding of where your Hudi jobs is spending its time, 
use a tool like [YourKit Java Profiler](https://www.yourkit.com/download/), to 
obtain heap dumps/flame graphs.
+:::
+
+Writing data via Hudi happens as a Spark job and thus general rules of spark 
debugging applies here too. Below is a list of things to keep in mind, if you 
are looking to improving performance or reliability.
+
+**Input Parallelism** : By default, Hudi tends to over-partition input (i.e 
`withParallelism(1500)`), to ensure each Spark partition stays within the 2GB 
limit for inputs upto 500GB. Bump this up accordingly if you have larger 
inputs. We recommend having shuffle parallelism 
`hoodie.[insert|upsert|bulkinsert].shuffle.parallelism` such that its atleast 
input_data_size/500MB
+
+**Off-heap memory** : Hudi writes parquet files and that needs good amount of 
off-heap memory proportional to schema width. Consider setting something like 
`spark.yarn.executor.memoryOverhead` or `spark.yarn.driver.memoryOverhead`, if 
you are running into such failures.
+
+**Spark Memory** : Typically, hudi needs to be able to read a single file into 
memory to perform merges or compactions and thus the executor memory should be 
sufficient to accomodate this. In addition, Hoodie caches the input to be able 
to intelligently place data and thus leaving some 
`spark.memory.storageFraction` will generally help boost performance.
+
+**Sizing files**: Set `limitFileSize` above judiciously, to balance 
ingest/write latency vs number of files & consequently metadata overhead 
associated with it.
+
+**Timeseries/Log data** : Default configs are tuned for database/nosql 
changelogs where individual record sizes are large. Another very popular class 
of data is timeseries/event/log data that tends to be more volumnious with lot 
more records per partition. In such cases consider tuning the bloom filter 
accuracy via `.bloomFilterFPP()/bloomFilterNumEntries()` to achieve your target 
index look up time. Also, consider making a key that is prefixed with time of 
the event, which will enable r [...]
+
+**GC Tuning**: Please be sure to follow garbage collection tuning tips from 
Spark tuning guide to avoid OutOfMemory errors. [Must] Use G1/CMS Collector. 
Sample CMS Flags to add to spark.executor.extraJavaOptions:
+
+```java
+-XX:NewSize=1g -XX:SurvivorRatio=2 -XX:+UseCompressedOops 
-XX:+UseConcMarkSweepGC -XX:+UseParNewGC -XX:CMSInitiatingOccupancyFraction=70 
-XX:+PrintGCDetails -XX:+PrintGCTimeStamps -XX:+PrintGCDateStamps 
-XX:+PrintGCApplicationStoppedTime -XX:+PrintGCApplicationConcurrentTime 
-XX:+PrintTenuringDistribution -XX:+HeapDumpOnOutOfMemoryError 
-XX:HeapDumpPath=/tmp/hoodie-heapdump.hprof
+```
+
+**OutOfMemory Errors**: If it keeps OOMing still, reduce spark memory 
conservatively: `spark.memory.fraction=0.2, spark.memory.storageFraction=0.2` 
allowing it to spill rather than OOM. (reliably slow vs crashing intermittently)
+
+Below is a full working production config
+
+```scala
+spark.driver.extraClassPath /etc/hive/conf
+spark.driver.extraJavaOptions -XX:+PrintTenuringDistribution 
-XX:+PrintGCDetails -XX:+PrintGCDateStamps -XX:+PrintGCApplicationStoppedTime 
-XX:+PrintGCApplicationConcurrentTime -XX:+PrintGCTimeStamps 
-XX:+HeapDumpOnOutOfMemoryError -XX:HeapDumpPath=/tmp/hoodie-heapdump.hprof
+spark.driver.maxResultSize 2g
+spark.driver.memory 4g
+spark.executor.cores 1
+spark.executor.extraJavaOptions -XX:+PrintFlagsFinal -XX:+PrintReferenceGC 
-verbose:gc -XX:+PrintGCDetails -XX:+PrintGCTimeStamps 
-XX:+PrintAdaptiveSizePolicy -XX:+UnlockDiagnosticVMOptions 
-XX:+HeapDumpOnOutOfMemoryError -XX:HeapDumpPath=/tmp/hoodie-heapdump.hprof
+spark.executor.id driver
+spark.executor.instances 300
+spark.executor.memory 6g
+spark.rdd.compress true
+ 
+spark.kryoserializer.buffer.max 512m
+spark.serializer org.apache.spark.serializer.KryoSerializer
+spark.shuffle.service.enabled true
+spark.sql.hive.convertMetastoreParquet false
+spark.submit.deployMode cluster
+spark.task.cpus 1
+spark.task.maxFailures 4
+ 
+spark.yarn.driver.memoryOverhead 1024
+spark.yarn.executor.memoryOverhead 3072
+spark.yarn.max.executor.failures 100
+```
\ No newline at end of file
diff --git a/website/sidebars.js b/website/sidebars.js
index a1dc301..4f435b5 100644
--- a/website/sidebars.js
+++ b/website/sidebars.js
@@ -71,6 +71,7 @@ module.exports = {
                 'cli',
                 'metrics',
                 'troubleshooting',
+                'tuning-guide',
                 {
                     type: 'category',
                     label: 'Storage Configurations',

Reply via email to