Migration Guide
Upgrading from 0.6 to 0.7
-
Since 0.7.0, Celeborn removed
ReleaseSlots. -
Since 0.7.0, Celeborn removed
WorkerRemove. -
Since 0.7.0, Celeborn worker metrics
FlushDataTimeis renamed asFlushLocalDataTime. -
Since 0.7.0, Celeborn changed the default value of
celeborn.<module>.io.modefromNIOtoKQUEUEif kqueue mode is available, falling back toNIOotherwise. -
Since 0.7.0, Celeborn changed the default value of
celeborn.worker.directMemoryRatioForReadBufferfrom0.1to0.35, which means read buffer threshold of buffer dispatcher is max direct memory * 0.35 at default.
Upgrading from 0.5 to 0.6
-
Since 0.6.0, Celeborn deprecate
celeborn.client.spark.fetch.throwsFetchFailure. Please useceleborn.client.spark.stageRerun.enabledinstead. -
Since 0.6.0, Celeborn modified
celeborn.quota.tenant.diskBytesWrittentoceleborn.quota.user.diskBytesWritten. Please useceleborn.quota.user.diskBytesWrittenif you want to set user level quota. -
Since 0.6.0, Celeborn modified
celeborn.quota.tenant.diskFileCounttoceleborn.quota.user.diskFileCount. Please useceleborn.quota.user.diskFileCountif you want to set user level quota. -
Since 0.6.0, Celeborn modified
celeborn.quota.tenant.hdfsBytesWrittentoceleborn.quota.user.hdfsBytesWritten. Please useceleborn.quota.user.hdfsBytesWrittenif you want to set user level quota. -
Since 0.6.0, Celeborn modified
celeborn.quota.tenant.hdfsFileCounttoceleborn.quota.user.hdfsFileCount. Please useceleborn.quota.user.hdfsFileCountif you want to set user level quota. -
Since 0.6.0, Celeborn modified
celeborn.master.hdfs.expireDirs.timeouttoceleborn.master.dfs.expireDirs.timeout. Please usecceleborn.master.dfs.expireDirs.timeoutif you want to set timeout for an expired dirs to be deleted. -
Since 0.6.0, Celeborn introduced
celeborn.master.slot.assign.minWorkerswith default value of100, which means Celeborn will involve more workers in offering slots when number of reducers are less. -
Since 0.6.0, Celeborn deprecate
celeborn.worker.congestionControl.low.watermark. Please useceleborn.worker.congestionControl.diskBuffer.low.watermarkinstead. -
Since 0.6.0, Celeborn deprecate
celeborn.worker.congestionControl.high.watermark. Please useceleborn.worker.congestionControl.diskBuffer.high.watermarkinstead. -
Since 0.6.0, Celeborn changed the default value of
celeborn.client.spark.fetch.throwsFetchFailurefromfalsetotrue, which means Celeborn will enable spark stage rerun at default. -
Since 0.6.0, Celeborn changed
celeborn.<module>.io.modeoptional, of which the default value changed fromNIOtoEPOLLif epoll mode is available, falling back toNIOotherwise. -
Since 0.6.0, Celeborn removed
celeborn.client.shuffle.mapPartition.split.enabledto enable shuffle partition split at default for MapPartition. -
Since 0.6.0, Celeborn has introduced a new RESTful API namespace: /api/v1, which uses the application/json media type for requests and responses. The
celeborn-openapi-clientSDK is also available to help users interact with the new RESTful APIs. The legacy RESTful APIs have been deprecated and will be removed in future releases. Access the full RESTful API documentation for detailed information. -
The mappings of the old RESTful APIs to the new RESTful APIs for Master.
Old RESTful API New RESTful API Note GET /conf GET /api/v1/conf GET /listDynamicConfigs GET /api/v1/conf/dynamic GET /threadDump GET /api/v1/thread_dump GET /applications GET /api/v1/applications GET /hostnames GET /api/v1/applications/hostnames GET /shuffle GET /api/v1/shuffles GET /masterGroupInfo GET /api/v1/masters GET /workerInfo GET /api/v1/workers GET /lostWorkers GET /api/v1/workers get the lostWorkers field in response GET /excludedWorkers GET /api/v1/workers get the excludedWorkers field in response GET /shutdownWorkers GET /api/v1/workers get the shutdownWorkers filed in response GET /decommissionWorkers GET /api/v1/workers get the decommissioningWorkers filed in response POST /exclude POST /api/v1/workers/exclude GET /workerEventInfo GET /api/v1/workers/events POST /sendWorkerEvent POST /api/v1/workers/events -
The mappings of the old RESTful APIs to the new RESTful APIs for Worker.
Old RESTful API New RESTful API Note GET /conf GET /api/v1/conf GET /listDynamicConfigs GET /api/v1/conf/dynamic GET /threadDump GET /api/v1/thread_dump GET /applications GET /api/v1/applications GET /shuffle GET /api/v1/shuffles GET /listPartitionLocationInfo GET /api/v1/shuffles/partitions GET /workerInfo GET /api/v1/workers GET /isRegistered GET /api/v1/workers get the isRegistered field in response GET /isDecommissioning GET /api/v1/workers get the isDecommissioning field in response GET /isShutdown GET /api/v1/workers get the isShutdown field in response GET /unavailablePeers GET /api/v1/workers/unavailable_peers POST /exit POST /api/v1/workers/exit -
Since 0.6.0, the RESTful api
/listTopDiskUsedAppsboth in Master and Worker has been removed. Please use the following PromQL query instead.topK(50, sum by (applicationId) (metrics_diskBytesWritten_Value{role="Worker", applicationId!=""})) -
Since 0.6.0, the out-of-dated Flink 1.14 and Flink 1.15 have been removed from the official support list.
-
Since 0.6.0, the client respects the spark.celeborn.storage.availableTypes configuration, ensuring revived partition locations no longer default to memory storage. In contrast, clients prior to 0.6.0 default to memory storage for revived partitions. This means that if memory storage is enabled in worker nodes, clients prior to 0.6.0 may inadvertently utilize memory storage for an application even when memory storage is not enabled for that app.
-
Since 0.6.0, we have added a new sink
org.apache.celeborn.common.metrics.sink.LoggerSinkto make sure that Celeborn metrics will be scraped periodically. It's recommended to enable this sink to make sure that worker's metrics data won't be too large to cause worker OOM if you don't have a collector to scrape metrics periodically. Don't forget to update themetrics.properties.
Upgrading from 0.5.0 to 0.5.1
-
Since 0.5.1, Celeborn master REST API
/excluderequest uses media typeapplication/x-www-form-urlencodedinstead oftext/plain. -
Since 0.5.1, Celeborn master REST API
/sendWorkerEventrequest uses POST method and the parameterstypeandworkersuse form parameters instead, and uses media typeapplication/x-www-form-urlencodedinstead oftext/plain. -
Since 0.5.1, Celeborn worker REST API
/exitrequest uses media typeapplication/x-www-form-urlencodedinstead oftext/plain.
Upgrading from 0.4 to 0.5
-
Since 0.5.0, Celeborn master metrics
LostWorkersis renamed asLostWorkerCount. -
Since 0.5.0, Celeborn worker metrics
ChunkStreamCountis renamed asActiveChunkStreamCount. -
Since 0.5.0, Celeborn worker metrics
CreditStreamCountis renamed asActiveCreditStreamCount. -
Since 0.5.0, Celeborn configurations support new tag
isDynamicto represent whether the configuration is dynamic config. -
Since 0.5.0, Celeborn changed the default value of
celeborn.worker.graceful.shutdown.recoverDbBackendfromLEVELDBtoROCKSDB, which means Celeborn will use RocksDB store for recovery backend. To restore the behavior before Celeborn 0.5, you can setceleborn.worker.graceful.shutdown.recoverDbBackendtoLEVELDB. -
Since 0.5.0, Celeborn deprecate
celeborn.quota.configuration.path. Please useceleborn.dynamicConfig.store.fs.pathinstead. -
Since 0.5.0, Celeborn client removes configuration
celeborn.client.push.splitPartition.threads,celeborn.client.flink.inputGate.minMemoryandceleborn.client.flink.resultPartition.minMemory. -
Since 0.5.0, Celeborn deprecate
celeborn.client.spark.shuffle.forceFallback.enabled. Please useceleborn.client.spark.shuffle.fallback.policyinstead. -
Since 0.5.0, Celeborn master REST API
/excludeuses POST method and the parametersaddandremoveuse form parameters instead. -
Since 0.5.0, Celeborn worker REST API
/exituses POST method and the parametertypeuses form parameter instead. -
Since 0.5.0, Celeborn master and worker REST API
/shufflesis renamed as/shuffle, and will be deprecated since 0.6.0.
Upgrading from 0.4.0 to 0.4.1
-
Since 0.4.1, Celeborn master adds a limit to the estimated partition size used for computing worker slots. This size is now constrained within the range specified by
celeborn.master.estimatedPartitionSize.minSizeandceleborn.master.estimatedPartitionSize.maxSize. -
Since 0.4.1, Celeborn changed the fallback configuration of
celeborn.client.rpc.getReducerFileGroup.askTimeout,celeborn.client.rpc.registerShuffle.askTimeoutandceleborn.client.rpc.requestPartition.askTimeoutfromceleborn.<module>.io.connectionTimeouttoceleborn.rpc.askTimeout.
Upgrading from 0.3 to 0.4
-
Since 0.4.0, Celeborn won't be compatible with Celeborn client that versions below 0.3.0. Note that: It's strongly recommended to use the same version of Client and Celeborn Master/Worker in production.
-
Since 0.4.0, Celeborn won't support
org.apache.spark.shuffle.celeborn.RssShuffleManager. -
Since 0.4.0, Celeborn changed the default value of
celeborn.<module>.io.numConnectionsPerPeerfrom2to1. -
Since 0.4.0, Celeborn has changed the names of the prometheus master and worker configuration as shown in the table below:
Key Before v0.4.0 Key After v0.4.0 celeborn.metrics.master.prometheus.hostceleborn.master.http.hostceleborn.metrics.master.prometheus.portceleborn.master.http.portceleborn.metrics.worker.prometheus.hostceleborn.worker.http.hostceleborn.metrics.worker.prometheus.portceleborn.worker.http.port -
Since 0.4.0, Celeborn deprecate
celeborn.worker.storage.baseDir.prefixandceleborn.worker.storage.baseDir.number. Please useceleborn.worker.storage.dirsinstead. -
Since 0.4.0, Celeborn deprecate
celeborn.storage.activeTypes. Please useceleborn.storage.availableTypesinstead. -
Since 0.4.0, Celeborn worker removes configuration
celeborn.worker.userResourceConsumption.update.interval. -
Since 0.4.0, Celeborn master metrics
PartitionWrittenis renamed asActiveShuffleSize. -
Since 0.4.0, Celeborn master metrics
PartitionFileCountis renamed asActiveShuffleFileCount.
Upgrading from 0.3.1 to 0.3.2
-
Since 0.3.1, Celeborn changed the default value of
raft.client.rpc.request.timeoutfrom3sto10s. -
Since 0.3.1, Celeborn changed the default value of
raft.client.rpc.watch.request.timeoutfrom10sto20s.
Upgrading from 0.3.0 to 0.3.1
-
Since 0.3.1, Celeborn changed the default value of
celeborn.worker.directMemoryRatioToResumefrom0.5to0.7. -
Since 0.3.1, Celeborn changed the default value of
celeborn.worker.monitor.disk.check.intervalfrom60to30. -
Since 0.3.1, name of JVM metrics changed, see details at CELEBORN-1007.
Upgrading from 0.2 to 0.3
-
Celeborn 0.2 Client is compatible with 0.3 Master/Server, it allows to upgrade Master/Worker first then Client. Note that: It's strongly recommended to use the same version of Client and Celeborn Master/Worker in production.
-
Since 0.3.0, the support of deprecated configurations
rss.*is removed. All configurations listed in 0.2.1 docs still take effect, but some of those are deprecated too, please read the bootstrap logs and follow the suggestion to migrate to the new configuration. -
From 0.3.0 on the default value for
celeborn.client.push.replicate.enabledis changed fromtruetofalse, users who want replication on should explicitly enable replication. For example, to enable replication for Spark users should add the spark config when submitting job:spark.celeborn.client.push.replicate.enabled=true -
From 0.3.0 on the default value for
celeborn.worker.storage.workingDiris changed fromhadoop/rss-worker/shuffle_datatoceleborn-worker/shuffle_data, users who want to use origin working dir path should set this configuration. -
Since 0.3.0, configuration namespace
celeborn.ha.masteris deprecated, and will be removed in the future versions. All configurationsceleborn.ha.master.*should migrate toceleborn.master.ha.*. -
Since 0.3.0, environment variables
CELEBORN_MASTER_HOSTandCELEBORN_MASTER_PORTare removed. InsteadCELEBORN_LOCAL_HOSTNAMEworks on both master and worker, which takes high priority than configurations defined in properties file. -
Since 0.3.0, the Celeborn Master URL schema is changed from
rss://toceleborn://, for users who start Worker bysbin/start-worker.sh rss://<master-host>:<master-port>, should migrate tosbin/start-worker.sh celeborn://<master-host>:<master-port>. -
Since 0.3.0, Celeborn supports overriding Hadoop configuration(
core-site.xml,hdfs-site.xml, etc.) from Celeborn configuration with the additional prefixceleborn.hadoop.. On Spark client side, user should set Hadoop configuration likespark.celeborn.hadoop.foo=bar, note thatspark.hadoop.foo=bardoes not take effect; on Flink client and Celeborn Master/Worker side, user should set likeceleborn.hadoop.foo=bar. -
Since 0.3.0, Celeborn master metrics
BlacklistedWorkerCountis renamed asExcludedWorkerCount. -
Since 0.3.0, Celeborn master http request url
/blacklistedWorkersis renamed as/excludedWorkers. -
Since 0.3.0, introduces a terminology update for Celeborn worker data replication, replacing the previous
master/slaveterminology withprimary/replica. In alignment with this change, corresponding metrics keywords have been adjusted. The following table presents a comprehensive overview of the changes:Key Before v0.3.0 Key After v0.3.0 MasterPushDataTimePrimaryPushDataTimeMasterPushDataHandshakeTimePrimaryPushDataHandshakeTimeMasterRegionStartTimePrimaryRegionStartTimeMasterRegionFinishTimePrimaryRegionFinishTimeSlavePushDataTimeReplicaPushDataTimeSlavePushDataHandshakeTimeReplicaPushDataHandshakeTimeSlaveRegionStartTimeReplicaRegionStartTimeSlaveRegionFinishTimeReplicaRegionFinishTime -
Since 0.3.0, Celeborn's spark shuffle manager change from
org.apache.spark.shuffle.celeborn.RssShuffleManagertoorg.apache.spark.shuffle.celeborn.SparkShuffleManager. User can set spark propertyspark.shuffle.managertoorg.apache.spark.shuffle.celeborn.SparkShuffleManagerto use Celeborn remote shuffle service. In 0.3.0, Celeborn still supportorg.apache.spark.shuffle.celeborn.RssShuffleManager, it will be removed in 0.4.0.